From 17842b498251e8a78148cc5cf85ab6ff55ddc9c2 Mon Sep 17 00:00:00 2001 From: mjsax Date: Tue, 15 Sep 2015 23:59:31 +0200 Subject: [PATCH 1/2] [FLINK-2566] FlinkTopologyContext not populated completely - extended FlinkTopologyContext to be populted with all supportable attributes - added JUnit test - updated README.md additionally: module restructuring to get cleaner package structure This closes #1135 --- .../flink-storm-compatibility-core/README.md | 2 +- .../stormcompatibility/api/FlinkTopology.java | 16 +- .../api/FlinkTopologyBuilder.java | 39 ++-- .../{wrappers => util}/FiniteStormSpout.java | 4 +- .../FlinkOutputFieldsDeclarer.java | 14 +- .../util/FlinkStormStreamSelector.java | 2 +- .../{api => util}/FlinkTopologyContext.java | 29 +-- .../util/SplitStreamTypeKeySelector.java | 46 +++++ .../wrappers/AbstractStormSpoutWrapper.java | 36 ++-- .../wrappers/FiniteStormSpoutWrapper.java | 1 + ...er.java => SetupOutputFieldsDeclarer.java} | 4 +- .../wrappers/StormBoltWrapper.java | 17 +- .../wrappers/StormWrapperSetupHelper.java | 192 +++++++++++++++-- .../api/FlinkTopologyContextTest.java | 74 ------- .../api/FlinkTopologyTest.java | 10 +- .../api/TestTopologyBuilder.java | 27 +++ .../{wrappers => util}/FiniteTestSpout.java | 4 +- .../FlinkOutputFieldsDeclarerTest.java | 4 +- .../FlinkStormStreamSelectorTest.java | 2 +- .../util/FlinkTopologyContextTest.java | 114 ++++++++++ .../util/TestDummyBolt.java | 20 +- .../util/TestDummySpout.java | 17 +- .../stormcompatibility/util/TestSink.java | 16 +- .../wrappers/FiniteStormSpoutWrapperTest.java | 6 + ...ava => SetupOutputFieldsDeclarerTest.java} | 18 +- .../wrappers/StormBoltWrapperTest.java | 36 ++-- .../wrappers/StormFiniteSpoutWrapperTest.java | 13 +- .../wrappers/StormSpoutWrapperTest.java | 6 + .../wrappers/StormWrapperSetupHelperTest.java | 194 +++++++++++++++++- .../util/FiniteStormFileSpout.java | 2 +- .../util/FiniteStormInMemorySpout.java | 2 +- .../split/SplitBoltTopology.java | 4 +- 32 files changed, 760 insertions(+), 211 deletions(-) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/{wrappers => util}/FiniteStormSpout.java (91%) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/{api => util}/FlinkOutputFieldsDeclarer.java (92%) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/{api => util}/FlinkTopologyContext.java (79%) create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/{StormOutputFieldsDeclarer.java => SetupOutputFieldsDeclarer.java} (94%) delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContextTest.java create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/{wrappers => util}/FiniteTestSpout.java (95%) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/{api => util}/FlinkOutputFieldsDeclarerTest.java (97%) rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/{wrappers => util}/FlinkStormStreamSelectorTest.java (97%) create mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java rename flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/{StormOutputFieldsDeclarerTest.java => SetupOutputFieldsDeclarerTest.java} (81%) diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md index f42dc24f9659f..9663fc714f9f3 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md @@ -3,7 +3,7 @@ The Storm compatibility layer allows to embed spouts or bolt unmodified within a regular Flink streaming program (`StormSpoutWrapper` and `StormBoltWrapper`). Additionally, a whole Storm topology can be submitted to Flink (see `FlinkTopologyBuilder`, `FlinkLocalCluster`, and `FlinkSubmitter`). Only a few minor changes to the original submitting code are required. The code that builds the topology itself, can be reused unmodified. See `flink-storm-examples` for a simple word-count example. The following Strom features are not (yet/fully) supported by the compatibility layer right now: -* topology and tuple meta information (ie, `TopologyContext` not fully supported) +* tuple meta information * no fault-tolerance guarantees (ie, calls to `ack()`/`fail()` and anchoring is ignored) * for whole Storm topologies the following is not supported by Flink: * direct emit connection pattern diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java index 8c75a2cac5094..179466ed66895 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java @@ -28,17 +28,14 @@ * cannot be executed directly, but must be handed over to a {@link FlinkLocalCluster}, {@link FlinkSubmitter}, or * {@link FlinkClient}. */ -class FlinkTopology extends StreamExecutionEnvironment { +public class FlinkTopology extends StreamExecutionEnvironment { - /** The corresponding {@link StormTopology} that is mimicked by this {@link FlinkTopology} */ - private final StormTopology stormTopology; /** The number of declared tasks for the whole program (ie, sum over all dops) */ private int numberOfTasks = 0; - public FlinkTopology(final StormTopology stormTopology) { + public FlinkTopology() { // Set default parallelism to 1, to mirror Storm default behavior super.setParallelism(1); - this.stormTopology = stormTopology; } /** @@ -52,7 +49,7 @@ public FlinkTopology(final StormTopology stormTopology) { public JobExecutionResult execute() throws Exception { throw new UnsupportedOperationException( "A FlinkTopology cannot be executed directly. Use FlinkLocalCluster, FlinkSubmitter, or FlinkClient " + - "instead."); + "instead."); } /** @@ -66,12 +63,7 @@ public JobExecutionResult execute() throws Exception { public JobExecutionResult execute(final String jobName) throws Exception { throw new UnsupportedOperationException( "A FlinkTopology cannot be executed directly. Use FlinkLocalCluster, FlinkSubmitter, or FlinkClient " + - "instead."); - } - - //TODO - public String getStormTopologyAsString() { - return this.stormTopology.toString(); + "instead."); } /** diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java index e4f6c94e0bcb0..d62d56b8d83a8 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java @@ -16,7 +16,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flink.stormcompatibility.api; import backtype.storm.generated.ComponentCommon; @@ -35,10 +34,12 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.stormcompatibility.util.FiniteStormSpout; +import org.apache.flink.stormcompatibility.util.FlinkOutputFieldsDeclarer; import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector; import org.apache.flink.stormcompatibility.util.SplitStreamType; +import org.apache.flink.stormcompatibility.util.SplitStreamTypeKeySelector; import org.apache.flink.stormcompatibility.wrappers.AbstractStormSpoutWrapper; -import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpout; import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; import org.apache.flink.stormcompatibility.wrappers.StormSpoutWrapper; @@ -73,14 +74,18 @@ public class FlinkTopologyBuilder { private final HashMap> outputStreams = new HashMap>(); /** All spouts&bolts declarers by their ID */ private final HashMap declarers = new HashMap(); + // needs to be a class member for internal testing purpose + private StormTopology stormTopology; + /** * Creates a Flink program that uses the specified spouts and bolts. */ @SuppressWarnings({"rawtypes", "unchecked"}) public FlinkTopology createTopology() { - final StormTopology stormTopology = this.stormBuilder.createTopology(); - final FlinkTopology env = new FlinkTopology(stormTopology); + this.stormTopology = this.stormBuilder.createTopology(); + + final FlinkTopology env = new FlinkTopology(); env.setParallelism(1); final HashMap> availableInputs = new HashMap>(); @@ -102,6 +107,7 @@ public FlinkTopology createTopology() { } else { spoutWrapper = new StormSpoutWrapper(userSpout); } + spoutWrapper.setStormTopology(stormTopology); DataStreamSource source; HashMap outputStreams = new HashMap(); @@ -126,6 +132,8 @@ public FlinkTopology createTopology() { if (common.is_set_parallelism_hint()) { dop = common.get_parallelism_hint(); source.setParallelism(dop); + } else { + common.set_parallelism_hint(1); } env.increaseNumberOfTasks(dop); } @@ -217,6 +225,7 @@ public FlinkTopology createTopology() { } SingleOutputStreamOperator outputStream; + StormBoltWrapper boltWrapper; if (boltOutputStreams.size() < 2) { // single output stream or sink String outputStreamId = null; if (boltOutputStreams.size() == 1) { @@ -225,11 +234,9 @@ public FlinkTopology createTopology() { final TypeInformation outType = declarer .getOutputType(outputStreamId); - outputStream = inputStream.transform( - boltId, - outType, - new StormBoltWrapper(userBolt, this.outputStreams.get( - producerId).get(inputStreamId))); + boltWrapper = new StormBoltWrapper(userBolt, this.outputStreams + .get(producerId).get(inputStreamId)); + outputStream = inputStream.transform(boltId, outType, boltWrapper); if (outType != null) { // only for non-sink nodes @@ -241,11 +248,8 @@ public FlinkTopology createTopology() { final TypeInformation outType = TypeExtractor .getForClass(SplitStreamType.class); - outputStream = inputStream.transform( - boltId, - outType, - new StormBoltWrapper(userBolt, this.outputStreams.get( - producerId).get(inputStreamId))); + boltWrapper = new StormBoltWrapper(userBolt, this.outputStreams.get(producerId).get(inputStreamId)); + outputStream = inputStream.transform(boltId, outType, boltWrapper); SplitStream splitStreams = outputStream .split(new FlinkStormStreamSelector()); @@ -256,11 +260,14 @@ public FlinkTopology createTopology() { } availableInputs.put(boltId, op); } + boltWrapper.setStormTopology(stormTopology); int dop = 1; if (common.is_set_parallelism_hint()) { dop = common.get_parallelism_hint(); outputStream.setParallelism(dop); + } else { + common.set_parallelism_hint(1); } env.increaseNumberOfTasks(dop); @@ -393,4 +400,8 @@ public SpoutDeclarer setSpout(final String id, final IRichSpout spout, final Num * } */ + // for internal testing purpose only + StormTopology getStormTopology() { + return this.stormTopology; + } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java similarity index 91% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java index 58a4f7a11d246..114fa7c0d97e3 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java @@ -15,7 +15,9 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.stormcompatibility.util; + +import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; import backtype.storm.topology.IRichSpout; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java similarity index 92% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java index e2f63320d550d..3eee8d6402956 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.stormcompatibility.util; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; @@ -30,16 +30,14 @@ /** * {@link FlinkOutputFieldsDeclarer} is used to get the declared output schema of a - * {@link backtype.storm.topology.IRichSpout spout} or {@link backtype.storm.topology.IRichBolt - * bolt}.
+ * {@link backtype.storm.topology.IRichSpout spout} or {@link backtype.storm.topology.IRichBolt bolt}.
*
- * CAUTION: Currently, Flink does only support the default output stream. Furthermore, - * direct emit is not supported. + * CAUTION: Flink does not support direct emit. */ -final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer { +public final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer { - /** the declared output streams and schemas */ - final HashMap outputStreams = new HashMap(); + /** The declared output streams and schemas. */ + public final HashMap outputStreams = new HashMap(); @Override public void declare(final Fields fields) { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java index 7ca45d6db2714..7e60a87560ced 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java @@ -27,7 +27,7 @@ /** * Used by {@link FlinkTopologyBuilder} to split multiple declared output streams within Flink. */ -final public class FlinkStormStreamSelector implements OutputSelector> { +public final class FlinkStormStreamSelector implements OutputSelector> { private static final long serialVersionUID = 2553423379715401023L; /** internal cache to avoid short living ArrayList objects. */ diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContext.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java similarity index 79% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContext.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java index a7616170d5bc5..14af8305e721f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContext.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.stormcompatibility.util; import backtype.storm.generated.StormTopology; import backtype.storm.hooks.ITaskHook; @@ -26,31 +26,34 @@ import backtype.storm.metric.api.ReducedMetric; import backtype.storm.state.ISubscribedState; import backtype.storm.task.TopologyContext; +import backtype.storm.tuple.Fields; import java.util.Collection; +import java.util.List; import java.util.Map; +import clojure.lang.Atom; + /** * {@link FlinkTopologyContext} is a {@link TopologyContext} that overwrites certain method that are not applicable when * a Storm topology is executed within Flink. */ -public class FlinkTopologyContext extends TopologyContext { +public final class FlinkTopologyContext extends TopologyContext { /** * Instantiates a new {@link FlinkTopologyContext} for a given Storm topology. The context object is instantiated * for each parallel task - * - * @param topology - * The Storm topology that is currently executed - * @param taskToComponents - * A map from task IDs to Component IDs - * @param taskId - * The ID of the task the context belongs to. */ - public FlinkTopologyContext(final StormTopology topology, final Map taskToComponents, - final Integer taskId) { - super(topology, null, taskToComponents, null, null, null, null, null, taskId, null, null, null, null, null, - null, null); + public FlinkTopologyContext(final StormTopology topology, @SuppressWarnings("rawtypes") final Map stormConf, + final Map taskToComponent, final Map> componentToSortedTasks, + final Map> componentToStreamToFields, final String stormId, final String codeDir, + final String pidDir, final Integer taskId, final Integer workerPort, final List workerTasks, + final Map defaultResources, final Map userResources, + final Map executorData, @SuppressWarnings("rawtypes") final Map registeredMetrics, + final Atom openOrPrepareWasCalled) { + super(topology, stormConf, taskToComponent, componentToSortedTasks, componentToStreamToFields, stormId, + codeDir, pidDir, taskId, workerPort, workerTasks, defaultResources, userResources, executorData, + registeredMetrics, openOrPrepareWasCalled); } /** diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java new file mode 100644 index 0000000000000..200f77241a965 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java @@ -0,0 +1,46 @@ +/* + * 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.stormcompatibility.util; + +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.streaming.util.keys.KeySelectorUtil; +import org.apache.flink.streaming.util.keys.KeySelectorUtil.ArrayKeySelector; + +/** + * {@link SplitStreamTypeKeySelector} is a specific grouping key selector for streams that are selected via + * {@link FlinkStormStreamSelector} from a Spout or Bolt that declares multiple output streams. + * + * It extracts the wrapped {@link Tuple} type from the {@link SplitStreamType} tuples and applies a regular + * {@link ArrayKeySelector} on it. + */ +public class SplitStreamTypeKeySelector implements KeySelector, Tuple> { + private static final long serialVersionUID = 4672434660037669254L; + + private final ArrayKeySelector selector; + + public SplitStreamTypeKeySelector(int... fields) { + this.selector = new KeySelectorUtil.ArrayKeySelector(fields); + } + + @Override + public Tuple getKey(SplitStreamType value) throws Exception { + return selector.getKey(value.value); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java index c531580106f58..ccd29bb112caf 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.HashMap; +import backtype.storm.generated.StormTopology; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.topology.IRichSpout; @@ -43,22 +44,16 @@ public abstract class AbstractStormSpoutWrapper extends RichParallelSourceFunction { private static final long serialVersionUID = 4993283609095408765L; - /** - * Number of attributes of the bolt's output tuples per stream. - */ + /** Number of attributes of the bolt's output tuples per stream. */ private final HashMap numberOfAttributes; - /** - * The wrapped Storm {@link IRichSpout spout}. - */ + /** The wrapped Storm {@link IRichSpout spout}. */ protected final IRichSpout spout; - /** - * The wrapper of the given Flink collector. - */ + /** The wrapper of the given Flink collector. */ protected StormSpoutCollector collector; - /** - * Indicates, if the source is still running or was canceled. - */ + /** Indicates, if the source is still running or was canceled. */ protected volatile boolean isRunning = true; + /** The original Storm topology. */ + protected StormTopology stormTopology; /** * Instantiates a new {@link AbstractStormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such @@ -98,6 +93,16 @@ public AbstractStormSpoutWrapper(final IRichSpout spout, this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(spout, rawOutputs); } + /** + * Sets the original Storm topology. + * + * @param stormTopology + * The original Storm topology. + */ + public void setStormTopology(StormTopology stormTopology) { + this.stormTopology = stormTopology; + } + @Override public final void run(final SourceContext ctx) throws Exception { this.collector = new StormSpoutCollector(this.numberOfAttributes, ctx); @@ -114,8 +119,11 @@ public final void run(final SourceContext ctx) throws Exception { } this.spout.open(stormConfig, - StormWrapperSetupHelper - .convertToTopologyContext((StreamingRuntimeContext) super.getRuntimeContext(), true), + StormWrapperSetupHelper.createTopologyContext( + (StreamingRuntimeContext) super.getRuntimeContext(), + this.spout, + this.stormTopology, + null), new SpoutOutputCollector(this.collector)); this.spout.activate(); this.execute(); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java index 1912afc26e957..f499ecc167708 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java @@ -22,6 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; +import org.apache.flink.stormcompatibility.util.FiniteStormSpout; import com.google.common.collect.Sets; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java similarity index 94% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java index f33d4d3447fb1..3cd27d4fb4b10 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java @@ -24,11 +24,11 @@ import backtype.storm.utils.Utils; /** - * {@link StormOutputFieldsDeclarer} is used by {@link StormWrapperSetupHelper} to determine the output streams and + * {@link SetupOutputFieldsDeclarer} is used by {@link StormWrapperSetupHelper} to determine the output streams and * number of attributes declared by the wrapped spout's or bolt's {@code declare(...)}/{@code declareStream(...)} * method. */ -class StormOutputFieldsDeclarer implements OutputFieldsDeclarer { +class SetupOutputFieldsDeclarer implements OutputFieldsDeclarer { /** The number of attributes for each declared stream by the wrapped operator. */ HashMap outputSchemas = new HashMap(); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java index 6b58b0a4928b7..715d6df814b99 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.HashMap; +import backtype.storm.generated.StormTopology; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IRichBolt; @@ -59,6 +60,8 @@ public class StormBoltWrapper extends AbstractStreamOperator imple private final HashMap numberOfAttributes; /** The schema (ie, ordered field names) of the input stream. */ private final Fields inputSchema; + /** The original Storm topology. */ + protected StormTopology stormTopology; /** * We have to use this because Operators must output @@ -193,12 +196,22 @@ public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema, this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(bolt, rawOutputs); } + /** + * Sets the original Storm topology. + * + * @param stormTopology + * The original Storm topology. + */ + public void setStormTopology(StormTopology stormTopology) { + this.stormTopology = stormTopology; + } + @Override public void open(final Configuration parameters) throws Exception { super.open(parameters); - final TopologyContext topologyContext = StormWrapperSetupHelper.convertToTopologyContext( - super.runtimeContext, false); + final TopologyContext topologyContext = StormWrapperSetupHelper.createTopologyContext( + super.runtimeContext, this.bolt, this.stormTopology, null); flinkCollector = new TimestampedCollector(output); OutputCollector stormCollector = null; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java index 75ab8e0d8c07f..891497e874caf 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java @@ -14,32 +14,43 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flink.stormcompatibility.wrappers; +import backtype.storm.Config; import backtype.storm.generated.Bolt; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.SpoutSpec; +import backtype.storm.generated.StateSpoutSpec; import backtype.storm.generated.StormTopology; +import backtype.storm.generated.StreamInfo; import backtype.storm.task.TopologyContext; import backtype.storm.topology.IComponent; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.IRichSpout; +import backtype.storm.tuple.Fields; -import org.apache.flink.stormcompatibility.api.FlinkTopologyContext; +import org.apache.flink.stormcompatibility.util.FlinkOutputFieldsDeclarer; +import org.apache.flink.stormcompatibility.util.FlinkTopologyContext; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; +import clojure.lang.Atom; + +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; /** - * {@link StormWrapperSetupHelper} is an helper class used by {@link AbstractStormSpoutWrapper} or + * {@link StormWrapperSetupHelper} is an helper class used by {@link AbstractStormSpoutWrapper} and * {@link StormBoltWrapper}. */ class StormWrapperSetupHelper { + /** The configuration key for the topology name. */ + final static String TOPOLOGY_NAME = "storm.topology.name"; + /** * Computes the number of output attributes used by a {@link AbstractStormSpoutWrapper} or {@link StormBoltWrapper} * per declared output stream. The number is {@code -1} for raw output type or a value within range [0;25] for @@ -60,7 +71,7 @@ class StormWrapperSetupHelper { public static HashMap getNumberOfAttributes(final IComponent spoutOrBolt, final Collection rawOutputs) throws IllegalArgumentException { - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); spoutOrBolt.declareOutputFields(declarer); for (Entry schema : declarer.outputSchemas.entrySet()) { @@ -84,27 +95,174 @@ public static HashMap getNumberOfAttributes(final IComponent sp return declarer.outputSchemas; } - // TODO - public static TopologyContext convertToTopologyContext(final StreamingRuntimeContext context, - final boolean spoutOrBolt) { - final Integer taskId = new Integer(1 + context.getIndexOfThisSubtask()); + /** Used to computed unique task IDs for a Storm topology. */ + private static int tid; + + /** + * Creates a {@link TopologyContext} for a Spout or Bolt instance (ie, Flink task / Storm executor). + * + * @param context + * The Flink runtime context. + * @param spoutOrBolt + * The Spout or Bolt this context is created for. + * @param stormTopology + * The original Storm topology. + * @param stormConfig + * The user provided configuration. + * @return The created {@link TopologyContext}. + */ + @SuppressWarnings({ "rawtypes", "unchecked" }) + public static synchronized TopologyContext createTopologyContext( + final StreamingRuntimeContext context, final IComponent spoutOrBolt, + StormTopology stormTopology, Map stormConfig) { + String operatorName = context.getTaskName(); + if (operatorName.startsWith("Source: ")) { + // prefix "Source: " is inserted by Flink sources by default -- need to get rid of it here + operatorName = operatorName.substring(8); + } + final int dop = context.getNumberOfParallelSubtasks(); final Map taskToComponents = new HashMap(); - taskToComponents.put(taskId, context.getTaskName()); + final Map> componentToSortedTasks = new HashMap>(); + final Map> componentToStreamToFields = new HashMap>(); + String stormId = (String) stormConfig.get(TOPOLOGY_NAME); + String codeDir = null; // not supported + String pidDir = null; // not supported + Integer taskId = null; + Integer workerPort = null; // not supported + List workerTasks = new ArrayList(); + final Map defaultResources = new HashMap(); + final Map userResources = new HashMap(); + final Map executorData = new HashMap(); + final Map registeredMetrics = new HashMap(); + Atom openOrPrepareWasCalled = null; - final ComponentCommon common = new ComponentCommon(); - common.set_parallelism_hint(context.getNumberOfParallelSubtasks()); + if (stormTopology == null) { + // embedded mode + ComponentCommon common = new ComponentCommon(); + common.set_parallelism_hint(dop); - final Map bolts = new HashMap(); - final Map spoutSpecs = new HashMap(); + HashMap spouts = new HashMap(); + HashMap bolts = new HashMap(); + if (spoutOrBolt instanceof IRichSpout) { + spouts.put(operatorName, new SpoutSpec(null, common)); + } else { + assert (spoutOrBolt instanceof IRichBolt); + bolts.put(operatorName, new Bolt(null, common)); + } + stormTopology = new StormTopology(spouts, bolts, new HashMap()); - if (spoutOrBolt) { - spoutSpecs.put(context.getTaskName(), new SpoutSpec(null, common)); + taskId = context.getIndexOfThisSubtask(); + + List sortedTasks = new ArrayList(dop); + for (int i = 1; i <= dop; ++i) { + taskToComponents.put(i, operatorName); + sortedTasks.add(i); + } + componentToSortedTasks.put(operatorName, sortedTasks); + + FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); + spoutOrBolt.declareOutputFields(declarer); + componentToStreamToFields.put(operatorName, declarer.outputStreams); } else { - bolts.put(context.getTaskName(), new Bolt(null, common)); + // whole topology is built (ie, FlinkTopologyBuilder is used) + Map spouts = stormTopology.get_spouts(); + Map bolts = stormTopology.get_bolts(); + Map stateSpouts = stormTopology.get_state_spouts(); + + tid = 1; + + for (Entry spout : spouts.entrySet()) { + Integer rc = processSingleOperator(spout.getKey(), spout.getValue().get_common(), + operatorName, context.getIndexOfThisSubtask(), dop, taskToComponents, + componentToSortedTasks, componentToStreamToFields); + if (rc != null) { + taskId = rc; + } + } + for (Entry bolt : bolts.entrySet()) { + Integer rc = processSingleOperator(bolt.getKey(), bolt.getValue().get_common(), + operatorName, context.getIndexOfThisSubtask(), dop, taskToComponents, + componentToSortedTasks, componentToStreamToFields); + if (rc != null) { + taskId = rc; + } + } + for (Entry stateSpout : stateSpouts.entrySet()) { + Integer rc = taskId = processSingleOperator(stateSpout.getKey(), stateSpout + .getValue().get_common(), operatorName, context.getIndexOfThisSubtask(), + dop, taskToComponents, componentToSortedTasks, componentToStreamToFields); + if (rc != null) { + taskId = rc; + } + } + assert (taskId != null); + } + + if (!stormConfig.containsKey(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS)) { + stormConfig.put(Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS, 30); // Storm default value + } + + return new FlinkTopologyContext(stormTopology, stormConfig, taskToComponents, + componentToSortedTasks, componentToStreamToFields, stormId, codeDir, pidDir, + taskId, workerPort, workerTasks, defaultResources, userResources, executorData, + registeredMetrics, openOrPrepareWasCalled); + } + + /** + * Sets up {@code taskToComponents}, {@code componentToSortedTasks}, and {@code componentToStreamToFields} for a + * single instance of a Spout or Bolt (ie, task or executor). Furthermore, is computes the unique task-id. + * + * @param componentId + * The ID of the Spout/Bolt in the topology. + * @param common + * The common operator object (that is all Spouts and Bolts have). + * @param operatorName + * The Flink operator name. + * @param index + * The index of the currently processed tasks with its operator. + * @param dop + * The parallelism of the operator. + * @param taskToComponents + * OUTPUT: A map from all task IDs of the topology to their component IDs. + * @param componentToSortedTasks + * OUTPUT: A map from all component IDs to their sorted list of corresponding task IDs. + * @param componentToStreamToFields + * OUTPUT: A map from all component IDs to there output streams and output fields. + * + * @return A unique task ID if the currently processed Spout or Bolt ({@code componentId}) is equal to the current + * Flink operator ({@link operatorName}) -- {@code null} otherwise. + */ + private static Integer processSingleOperator(final String componentId, + final ComponentCommon common, final String operatorName, final int index, + final int dop, final Map taskToComponents, + final Map> componentToSortedTasks, + final Map> componentToStreamToFields) { + final int parallelism_hint = common.get_parallelism_hint(); + Integer taskId = null; + + if (componentId.equals(operatorName)) { + taskId = tid + index; + } + + List sortedTasks = new ArrayList(dop); + for (int i = 0; i < parallelism_hint; ++i) { + taskToComponents.put(tid, componentId); + sortedTasks.add(tid); + ++tid; + } + componentToSortedTasks.put(componentId, sortedTasks); + + if (componentId.equals(operatorName)) { + } + + Map outputStreams = new HashMap(); + for(Entry outStream : common.get_streams().entrySet()) { + outputStreams.put(outStream.getKey(), new Fields(outStream.getValue().get_output_fields())); } + componentToStreamToFields.put(componentId, outputStreams); - return new FlinkTopologyContext(new StormTopology(spoutSpecs, bolts, null), taskToComponents, taskId); + return taskId; } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContextTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContextTest.java deleted file mode 100644 index d214610d08ade..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContextTest.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.stormcompatibility.api; - -import backtype.storm.metric.api.ICombiner; -import backtype.storm.metric.api.IMetric; -import backtype.storm.metric.api.IReducer; -import org.junit.Test; - -public class FlinkTopologyContextTest { - - @Test(expected = UnsupportedOperationException.class) - public void testAddTaskHook() { - new FlinkTopologyContext(null, null, null).addTaskHook(null); - } - - @Test(expected = UnsupportedOperationException.class) - public void testGetHooks() { - new FlinkTopologyContext(null, null, null).getHooks(); - } - - @SuppressWarnings("rawtypes") - @Test(expected = UnsupportedOperationException.class) - public void testRegisteredMetric1() { - new FlinkTopologyContext(null, null, null).registerMetric(null, (ICombiner) null, 0); - } - - @SuppressWarnings("rawtypes") - @Test(expected = UnsupportedOperationException.class) - public void testRegisteredMetric2() { - new FlinkTopologyContext(null, null, null).registerMetric(null, (IReducer) null, 0); - } - - @Test(expected = UnsupportedOperationException.class) - public void testRegisteredMetric3() { - new FlinkTopologyContext(null, null, null).registerMetric(null, (IMetric) null, 0); - } - - @Test(expected = UnsupportedOperationException.class) - public void testGetRegisteredMetricByName() { - new FlinkTopologyContext(null, null, null).getRegisteredMetricByName(null); - } - - @Test(expected = UnsupportedOperationException.class) - public void testSetAllSubscribedState() { - new FlinkTopologyContext(null, null, null).setAllSubscribedState(null); - } - - @Test(expected = UnsupportedOperationException.class) - public void testSetSubscribedState1() { - new FlinkTopologyContext(null, null, null).setSubscribedState(null, null); - } - - @Test(expected = UnsupportedOperationException.class) - public void testSetSubscribedState2() { - new FlinkTopologyContext(null, null, null).setSubscribedState(null, null, null); - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java index f179919180e55..c98c9a3af0bac 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java @@ -24,23 +24,23 @@ public class FlinkTopologyTest { @Test public void testDefaultParallelism() { - final FlinkTopology topology = new FlinkTopology(null); + final FlinkTopology topology = new FlinkTopology(); Assert.assertEquals(1, topology.getParallelism()); } @Test(expected = UnsupportedOperationException.class) public void testExecute() throws Exception { - new FlinkTopology(null).execute(); + new FlinkTopology().execute(); } @Test(expected = UnsupportedOperationException.class) public void testExecuteWithName() throws Exception { - new FlinkTopology(null).execute(null); + new FlinkTopology().execute(null); } @Test public void testNumberOfTasks() { - final FlinkTopology topology = new FlinkTopology(null); + final FlinkTopology topology = new FlinkTopology(); Assert.assertEquals(0, topology.getNumberOfTasks()); @@ -56,7 +56,7 @@ public void testNumberOfTasks() { @Test(expected = AssertionError.class) public void testAssert() { - new FlinkTopology(null).increaseNumberOfTasks(0); + new FlinkTopology().increaseNumberOfTasks(0); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java new file mode 100644 index 0000000000000..f664e584472ca --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java @@ -0,0 +1,27 @@ +/* + * 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.stormcompatibility.api; + +import backtype.storm.generated.StormTopology; + +public class TestTopologyBuilder extends FlinkTopologyBuilder { + @Override + public StormTopology getStormTopology() { + return super.getStormTopology(); + } +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteTestSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteTestSpout.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java index eef35cfdaff39..18918735beaef 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteTestSpout.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.stormcompatibility.util; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; @@ -26,7 +26,7 @@ import java.util.Map; -class FiniteTestSpout implements IRichSpout { +public class FiniteTestSpout implements IRichSpout { private static final long serialVersionUID = 7992419478267824279L; private int numberOfOutputTuples; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java index 08ac60bf69315..8e6356387429b 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.stormcompatibility.util; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.stormcompatibility.util.FlinkOutputFieldsDeclarer; import org.junit.Assert; import org.junit.Test; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java index c0a6ed341cd33..c3cb7d7b7629f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.stormcompatibility.util; import java.util.Iterator; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java new file mode 100644 index 0000000000000..bd9ea3f307c98 --- /dev/null +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java @@ -0,0 +1,114 @@ +/* + * 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.stormcompatibility.util; + +import java.util.HashMap; + +import backtype.storm.generated.Bolt; +import backtype.storm.generated.SpoutSpec; +import backtype.storm.generated.StateSpoutSpec; +import backtype.storm.generated.StormTopology; +import backtype.storm.metric.api.ICombiner; +import backtype.storm.metric.api.IMetric; +import backtype.storm.metric.api.IReducer; + +import org.apache.flink.stormcompatibility.util.FlinkTopologyContext; +import org.junit.Test; + + +/* + * FlinkTopologyContext.getSources(componentId) and FlinkTopologyContext.getTargets(componentId) are not tested here, + * because those are tested in StormWrapperSetupHelperTest. + */ +public class FlinkTopologyContextTest extends AbstractTest { + + @Test(expected = UnsupportedOperationException.class) + public void testAddTaskHook() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .addTaskHook(null); + } + + @Test(expected = UnsupportedOperationException.class) + public void testGetHooks() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .getHooks(); + } + + @SuppressWarnings("rawtypes") + @Test(expected = UnsupportedOperationException.class) + public void testRegisteredMetric1() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .registerMetric(null, (ICombiner) null, 0); + } + + @SuppressWarnings("rawtypes") + @Test(expected = UnsupportedOperationException.class) + public void testRegisteredMetric2() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .registerMetric(null, (IReducer) null, 0); + } + + @Test(expected = UnsupportedOperationException.class) + public void testRegisteredMetric3() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .registerMetric(null, (IMetric) null, 0); + } + + @Test(expected = UnsupportedOperationException.class) + public void testGetRegisteredMetricByName() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .getRegisteredMetricByName(null); + } + + @Test(expected = UnsupportedOperationException.class) + public void testSetAllSubscribedState() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .setAllSubscribedState(null); + } + + @Test(expected = UnsupportedOperationException.class) + public void testSetSubscribedState1() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .setSubscribedState(null, null); + } + + @Test(expected = UnsupportedOperationException.class) + public void testSetSubscribedState2() { + new FlinkTopologyContext(new StormTopology(new HashMap(), + new HashMap(), new HashMap()), null, null, + null, null, null, null, null, null, null, null, null, null, null, null, null) + .setSubscribedState(null, null, null); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java index ec487196a4539..b4993733a8320 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java @@ -24,6 +24,7 @@ import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; import backtype.storm.tuple.Tuple; +import backtype.storm.tuple.Values; public class TestDummyBolt implements IRichBolt { private static final long serialVersionUID = 6893611247443121322L; @@ -31,12 +32,27 @@ public class TestDummyBolt implements IRichBolt { public final static String shuffleStreamId = "shuffleStream"; public final static String groupingStreamId = "groupingStream"; + private boolean emit = true; + private TopologyContext context; + private OutputCollector collector; + @SuppressWarnings("rawtypes") @Override - public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {} + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + this.context = context; + this.collector = collector; + } @Override - public void execute(Tuple input) {} + public void execute(Tuple input) { + if (this.context.getThisTaskIndex() == 0) { + this.collector.emit(shuffleStreamId, input.getValues()); + } + if (this.emit) { + this.collector.emit(groupingStreamId, new Values("bolt", this.context)); + this.emit = false; + } + } @Override public void cleanup() {} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java index 62705b8d96a2e..345ca1224ec18 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java @@ -23,6 +23,7 @@ import backtype.storm.topology.IRichSpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; +import backtype.storm.tuple.Values; import backtype.storm.utils.Utils; public class TestDummySpout implements IRichSpout { @@ -30,9 +31,16 @@ public class TestDummySpout implements IRichSpout { public final static String spoutStreamId = "spout-stream"; + private boolean emit = true; + private TopologyContext context; + private SpoutOutputCollector collector; + @SuppressWarnings("rawtypes") @Override - public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {} + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + this.context = context; + this.collector = collector; + } @Override public void close() {} @@ -44,7 +52,12 @@ public void activate() {} public void deactivate() {} @Override - public void nextTuple() {} + public void nextTuple() { + if (this.emit) { + this.collector.emit(new Values(this.context)); + this.emit = false; + } + } @Override public void ack(Object msgId) {} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java index 5699219a3167f..c8e5584b5b240 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java @@ -16,6 +16,8 @@ */ package org.apache.flink.stormcompatibility.util; +import java.util.LinkedList; +import java.util.List; import java.util.Map; import backtype.storm.task.OutputCollector; @@ -27,12 +29,22 @@ public class TestSink implements IRichBolt { private static final long serialVersionUID = 4314871456719370877L; + public final static List result = new LinkedList(); + @SuppressWarnings("rawtypes") @Override - public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {} + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + result.add(context); + } @Override - public void execute(Tuple input) {} + public void execute(Tuple input) { + if (input.size() == 1) { + result.add((TopologyContext) input.getValue(0)); + } else { + result.add((TopologyContext) input.getValue(1)); + } + } @Override public void cleanup() {} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java index 381e13054aa6e..b44e8a1292375 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java @@ -18,6 +18,8 @@ package org.apache.flink.stormcompatibility.wrappers; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.stormcompatibility.util.FiniteStormSpout; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; import org.junit.Test; @@ -43,6 +45,8 @@ public void runAndExecuteTest1() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final FiniteStormSpoutWrapper wrapper = new FiniteStormSpoutWrapper(stormSpout); wrapper.setRuntimeContext(taskContext); @@ -59,6 +63,8 @@ public void runAndExecuteTest2() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final FiniteStormSpoutWrapper wrapper = new FiniteStormSpoutWrapper(stormSpout); wrapper.setRuntimeContext(taskContext); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java similarity index 81% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java rename to flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java index 561939f6516d1..738eb1e9fa63b 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java @@ -26,11 +26,11 @@ import java.util.ArrayList; -public class StormOutputFieldsDeclarerTest extends AbstractTest { +public class SetupOutputFieldsDeclarerTest extends AbstractTest { @Test public void testDeclare() { - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); int numberOfAttributes = this.r.nextInt(26); declarer.declare(createSchema(numberOfAttributes)); @@ -55,37 +55,37 @@ private Fields createSchema(final int numberOfAttributes) { @Test public void testDeclareDirect() { - new StormOutputFieldsDeclarer().declare(false, new Fields()); + new SetupOutputFieldsDeclarer().declare(false, new Fields()); } @Test(expected = UnsupportedOperationException.class) public void testDeclareDirectFail() { - new StormOutputFieldsDeclarer().declare(true, new Fields()); + new SetupOutputFieldsDeclarer().declare(true, new Fields()); } @Test public void testDeclareStream() { - new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, new Fields()); + new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, new Fields()); } @Test(expected = IllegalArgumentException.class) public void testDeclareStreamFail() { - new StormOutputFieldsDeclarer().declareStream(null, new Fields()); + new SetupOutputFieldsDeclarer().declareStream(null, new Fields()); } @Test public void testDeclareFullStream() { - new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields()); + new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields()); } @Test(expected = IllegalArgumentException.class) public void testDeclareFullStreamFailNonDefaultStream() { - new StormOutputFieldsDeclarer().declareStream(null, false, new Fields()); + new SetupOutputFieldsDeclarer().declareStream(null, false, new Fields()); } @Test(expected = UnsupportedOperationException.class) public void testDeclareFullStreamFailDirect() { - new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, new Fields()); + new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, new Fields()); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java index 5cfb1515cbc73..68175937b0202 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java @@ -61,9 +61,9 @@ public class StormBoltWrapperTest extends AbstractTest { @Test(expected = IllegalArgumentException.class) public void testWrapperRawType() throws Exception { - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields("dummy1", "dummy2")); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); new StormBoltWrapper(mock(IRichBolt.class), new String[] { Utils.DEFAULT_STREAM_ID }); @@ -71,26 +71,26 @@ public void testWrapperRawType() throws Exception { @Test(expected = IllegalArgumentException.class) public void testWrapperToManyAttributes1() throws Exception { - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); final String[] schema = new String[26]; for (int i = 0; i < schema.length; ++i) { schema[i] = "a" + i; } declarer.declare(new Fields(schema)); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); new StormBoltWrapper(mock(IRichBolt.class)); } @Test(expected = IllegalArgumentException.class) public void testWrapperToManyAttributes2() throws Exception { - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); final String[] schema = new String[26]; for (int i = 0; i < schema.length; ++i) { schema[i] = "a" + i; } declarer.declare(new Fields(schema)); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); new StormBoltWrapper(mock(IRichBolt.class), new String[] {}); } @@ -133,12 +133,14 @@ private void testWrapper(final int numberOfAttributes) throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final IRichBolt bolt = mock(IRichBolt.class); - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields(schema)); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, (Fields) null); wrapper.setup(mock(Output.class), taskContext); @@ -163,6 +165,8 @@ public void testMultipleOutputStreams() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final Output output = mock(Output.class); @@ -209,14 +213,17 @@ public void testOpen() throws Exception { final ExecutionConfig taskConfig = mock(ExecutionConfig.class); when(taskConfig.getGlobalJobParameters()).thenReturn(null).thenReturn(stormConfig) - .thenReturn(flinkConfig); + .thenReturn(flinkConfig); final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(taskConfig); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields("dummy")); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + final IRichBolt bolt = mock(IRichBolt.class); final StormBoltWrapper wrapper = new StormBoltWrapper(bolt); @@ -249,8 +256,11 @@ public void testOpenSink() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(taskConfig); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final IRichBolt bolt = mock(IRichBolt.class); + final StormBoltWrapper wrapper = new StormBoltWrapper(bolt); wrapper.setup(mock(Output.class), taskContext); @@ -275,9 +285,9 @@ public void testOpenSink() throws Exception { public void testClose() throws Exception { final IRichBolt bolt = mock(IRichBolt.class); - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields("dummy")); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); final StormBoltWrapper wrapper = new StormBoltWrapper(bolt); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java index a4eea7e37f4a8..77f1b05954a94 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java @@ -22,7 +22,9 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.configuration.Configuration; import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.stormcompatibility.util.FiniteTestSpout; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; import org.junit.Assert; @@ -46,12 +48,14 @@ public class StormFiniteSpoutWrapperTest extends AbstractTest { @SuppressWarnings("unchecked") @Test public void testRunExecuteFixedNumber() throws Exception { - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields("dummy")); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final IRichSpout spout = mock(IRichSpout.class); final int numberOfCalls = this.r.nextInt(50); @@ -73,6 +77,8 @@ public void testRunExecute() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final IRichSpout spout = new FiniteTestSpout(numberOfCalls); final StormFiniteSpoutWrapper> spoutWrapper = new StormFiniteSpoutWrapper>( @@ -94,11 +100,12 @@ public void testCancel() throws Exception { StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final IRichSpout spout = new FiniteTestSpout(numberOfCalls); final StormFiniteSpoutWrapper> spoutWrapper = new StormFiniteSpoutWrapper>( spout); - spoutWrapper.setRuntimeContext(taskContext); spoutWrapper.cancel(); final TestContext collector = new TestContext(); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java index 04dc48d86ba98..f4fb4daacf60d 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java @@ -25,9 +25,11 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.Configuration; import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.stormcompatibility.util.FiniteTestSpout; import org.apache.flink.stormcompatibility.util.StormConfig; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; + import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -89,8 +91,12 @@ public void testRunExecuteCancelInfinite() throws Exception { final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); final IRichSpout spout = new FiniteTestSpout(numberOfCalls); + + final StormSpoutWrapper> spoutWrapper = new StormSpoutWrapper>(spout); spoutWrapper.setRuntimeContext(taskContext); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java index 7497ffc2650bf..c799d63282037 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java @@ -14,29 +14,46 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flink.stormcompatibility.wrappers; import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import backtype.storm.Config; +import backtype.storm.LocalCluster; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.StormTopology; +import backtype.storm.task.TopologyContext; import backtype.storm.topology.IComponent; import backtype.storm.topology.IRichBolt; import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.TopologyBuilder; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; +import org.apache.flink.stormcompatibility.api.TestTopologyBuilder; import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.stormcompatibility.util.TestDummyBolt; +import org.apache.flink.stormcompatibility.util.TestDummySpout; +import org.apache.flink.stormcompatibility.util.TestSink; +import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; import com.google.common.collect.Sets; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +@PowerMockIgnore("javax.*") @RunWith(PowerMockRunner.class) @PrepareForTest(StormWrapperSetupHelper.class) public class StormWrapperSetupHelperTest extends AbstractTest { @@ -65,9 +82,9 @@ public void testRawType() throws Exception { boltOrSpout = mock(IRichBolt.class); } - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields("dummy1", "dummy2")); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, Sets.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID })); @@ -83,13 +100,13 @@ public void testToManyAttributes() throws Exception { boltOrSpout = mock(IRichBolt.class); } - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); final String[] schema = new String[26]; for (int i = 0; i < schema.length; ++i) { schema[i] = "a" + i; } declarer.declare(new Fields(schema)); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null); } @@ -119,9 +136,9 @@ private void testTupleTypes(final int numberOfAttributes) throws Exception { boltOrSpout = mock(IRichBolt.class); } - final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer(); + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); declarer.declare(new Fields(schema)); - PowerMockito.whenNew(StormOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); HashMap attributes = new HashMap(); attributes.put(Utils.DEFAULT_STREAM_ID, numberOfAttributes); @@ -132,4 +149,167 @@ private void testTupleTypes(final int numberOfAttributes) throws Exception { .newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }) : null)); } + @Test + public void testCreateTopologyContext() { + HashMap dops = new HashMap(); + dops.put("spout1", 1); + dops.put("spout2", 3); + dops.put("bolt1", 1); + dops.put("bolt2", 2); + dops.put("sink", 1); + + HashMap taskCounter = new HashMap(); + taskCounter.put("spout1", 0); + taskCounter.put("spout2", 0); + taskCounter.put("bolt1", 0); + taskCounter.put("bolt2", 0); + taskCounter.put("sink", 0); + + HashMap operators = new HashMap(); + operators.put("spout1", new TestDummySpout()); + operators.put("spout2", new TestDummySpout()); + operators.put("bolt1", new TestDummyBolt()); + operators.put("bolt2", new TestDummyBolt()); + operators.put("sink", new TestSink()); + + TopologyBuilder builder = new TopologyBuilder(); + + builder.setSpout("spout1", (IRichSpout) operators.get("spout1"), dops.get("spout1")); + builder.setSpout("spout2", (IRichSpout) operators.get("spout2"), dops.get("spout2")); + builder.setBolt("bolt1", (IRichBolt) operators.get("bolt1"), dops.get("bolt1")).shuffleGrouping("spout1"); + builder.setBolt("bolt2", (IRichBolt) operators.get("bolt2"), dops.get("bolt2")).allGrouping("spout2"); + builder.setBolt("sink", (IRichBolt) operators.get("sink"), dops.get("sink")) + .fieldsGrouping("bolt1", TestDummyBolt.groupingStreamId, new Fields("id")) + .shuffleGrouping("bolt1", TestDummyBolt.shuffleStreamId) + .fieldsGrouping("bolt2", TestDummyBolt.groupingStreamId, new Fields("id")) + .shuffleGrouping("bolt2", TestDummyBolt.shuffleStreamId); + + final int maxRetry = 3; + int counter; + for (counter = 0; counter < maxRetry; ++counter) { + LocalCluster cluster = new LocalCluster(); + Config c = new Config(); + c.setNumAckers(0); + cluster.submitTopology("test", c, builder.createTopology()); + Utils.sleep((counter + 1) * 5000); + cluster.shutdown(); + + if (TestSink.result.size() == 8) { + break; + } + } + Assert.assertTrue(counter < maxRetry); + + TestTopologyBuilder flinkBuilder = new TestTopologyBuilder(); + + flinkBuilder.setSpout("spout1", (IRichSpout) operators.get("spout1"), dops.get("spout1")); + flinkBuilder.setSpout("spout2", (IRichSpout) operators.get("spout2"), dops.get("spout2")); + flinkBuilder.setBolt("bolt1", (IRichBolt) operators.get("bolt1"), dops.get("bolt1")).shuffleGrouping("spout1"); + flinkBuilder.setBolt("bolt2", (IRichBolt) operators.get("bolt2"), dops.get("bolt2")).allGrouping("spout2"); + flinkBuilder.setBolt("sink", (IRichBolt) operators.get("sink"), dops.get("sink")) + .fieldsGrouping("bolt1", TestDummyBolt.groupingStreamId, new Fields("id")) + .shuffleGrouping("bolt1", TestDummyBolt.shuffleStreamId) + .fieldsGrouping("bolt2", TestDummyBolt.groupingStreamId, new Fields("id")) + .shuffleGrouping("bolt2", TestDummyBolt.shuffleStreamId); + + flinkBuilder.createTopology(); + StormTopology stormTopology = flinkBuilder.getStormTopology(); + + Set taskIds = new HashSet(); + + for (TopologyContext expectedContext : TestSink.result) { + final String thisComponentId = expectedContext.getThisComponentId(); + int index = taskCounter.get(thisComponentId); + + StreamingRuntimeContext context = mock(StreamingRuntimeContext.class); + when(context.getTaskName()).thenReturn(thisComponentId); + when(context.getNumberOfParallelSubtasks()).thenReturn(dops.get(thisComponentId)); + when(context.getIndexOfThisSubtask()).thenReturn(index); + taskCounter.put(thisComponentId, ++index); + + Config stormConfig = new Config(); + stormConfig.put(StormWrapperSetupHelper.TOPOLOGY_NAME, "test"); + + TopologyContext topologyContext = StormWrapperSetupHelper.createTopologyContext( + context, operators.get(thisComponentId), stormTopology, stormConfig); + + ComponentCommon expcetedCommon = expectedContext.getComponentCommon(thisComponentId); + ComponentCommon common = topologyContext.getComponentCommon(thisComponentId); + + Assert.assertNull(topologyContext.getCodeDir()); + Assert.assertNull(common.get_json_conf()); + Assert.assertNull(topologyContext.getExecutorData(null)); + Assert.assertNull(topologyContext.getPIDDir()); + Assert.assertNull(topologyContext.getResource(null)); + Assert.assertNull(topologyContext.getSharedExecutor()); + Assert.assertNull(expectedContext.getTaskData(null)); + Assert.assertNull(topologyContext.getThisWorkerPort()); + + Assert.assertTrue(expectedContext.getStormId().startsWith(topologyContext.getStormId())); + + Assert.assertEquals(expcetedCommon.get_inputs(), common.get_inputs()); + Assert.assertEquals(expcetedCommon.get_parallelism_hint(), common.get_parallelism_hint()); + Assert.assertEquals(expcetedCommon.get_streams(), common.get_streams()); + Assert.assertEquals(expectedContext.getComponentIds(), topologyContext.getComponentIds()); + Assert.assertEquals(expectedContext.getComponentStreams(thisComponentId), + topologyContext.getComponentStreams(thisComponentId)); + Assert.assertEquals(thisComponentId, topologyContext.getThisComponentId()); + Assert.assertEquals(expectedContext.getThisSources(), topologyContext.getThisSources()); + Assert.assertEquals(expectedContext.getThisStreams(), topologyContext.getThisStreams()); + Assert.assertEquals(expectedContext.getThisTargets(), topologyContext.getThisTargets()); + Assert.assertEquals(0, topologyContext.getThisWorkerTasks().size()); + + for (int taskId : topologyContext.getComponentTasks(thisComponentId)) { + Assert.assertEquals(thisComponentId, topologyContext.getComponentId(taskId)); + } + + for (String componentId : expectedContext.getComponentIds()) { + Assert.assertEquals(expectedContext.getSources(componentId), + topologyContext.getSources(componentId)); + Assert.assertEquals(expectedContext.getTargets(componentId), + topologyContext.getTargets(componentId)); + + for (String streamId : expectedContext.getComponentStreams(componentId)) { + Assert.assertEquals( + expectedContext.getComponentOutputFields(componentId, streamId).toList(), + topologyContext.getComponentOutputFields(componentId, streamId).toList()); + } + } + + for (String streamId : expectedContext.getThisStreams()) { + Assert.assertEquals(expectedContext.getThisOutputFields(streamId).toList(), + topologyContext.getThisOutputFields(streamId).toList()); + } + + HashMap taskToComponents = new HashMap(); + Set allTaskIds = new HashSet(); + for (String componentId : expectedContext.getComponentIds()) { + List possibleTasks = expectedContext.getComponentTasks(componentId); + List tasks = topologyContext.getComponentTasks(componentId); + + Iterator p_it = possibleTasks.iterator(); + Iterator t_it = tasks.iterator(); + while(p_it.hasNext()) { + Assert.assertTrue(t_it.hasNext()); + Assert.assertNull(taskToComponents.put(p_it.next(), componentId)); + Assert.assertTrue(allTaskIds.add(t_it.next())); + } + Assert.assertFalse(t_it.hasNext()); + } + + Assert.assertEquals(taskToComponents, expectedContext.getTaskToComponent()); + Assert.assertTrue(taskIds.add(topologyContext.getThisTaskId())); + + try { + topologyContext.getHooks(); + Assert.fail(); + } catch (UnsupportedOperationException e) { /* expected */ } + + try { + topologyContext.getRegisteredMetricByName(null); + Assert.fail(); + } catch (UnsupportedOperationException e) { /* expected */ } + } + } + } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java index 64b3e28120db8..d3776fb47311f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java @@ -20,7 +20,7 @@ import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Values; -import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpout; +import org.apache.flink.stormcompatibility.util.FiniteStormSpout; import java.io.IOException; import java.util.Map; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java index 6fb764d6c2793..5efff66b90494 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java @@ -18,7 +18,7 @@ package org.apache.flink.stormcompatibility.util; -import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpout; +import org.apache.flink.stormcompatibility.util.FiniteStormSpout; /** * Implements a Storm Spout that reads String[] data stored in the memory. The spout stops diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java index c992b6b4ebecc..5f637d3d98161 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java +++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java @@ -49,10 +49,10 @@ public static FlinkTopologyBuilder buildTopology() { final String[] tokens = outputPath.split(":"); final String outputFile = tokens[tokens.length - 1]; builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)) - .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); } else { builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4) - .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); } return builder; From ca1a6c143e24f01b471a110868e31fc11b64f961 Mon Sep 17 00:00:00 2001 From: mjsax Date: Fri, 2 Oct 2015 18:43:38 +0200 Subject: [PATCH 2/2] [Storm Compatibility] Maven module restucturing and cleanup - removed storm-parent; renamed storm-core and storm-examples - updated internal Java package structure * renamed package "stormcompatibility" to "storm" * unified *SpoutWrapper* to single SpoutWrapper.java class * moved classes to appropriate packages * shortened class names by stripping "Storm" - some more minor fixes, cleanups, and test improvements - updated READMEs and web documentation - updated examples pom.xml to assembly WordCount jars correctly --- docs/apis/storm_compatibility.md | 111 +++---- .../flink-storm-compatibility-core/README.md | 12 - .../api/SplitStreamTypeKeySelector.java | 47 --- .../wrappers/AbstractStormSpoutWrapper.java | 153 ---------- .../wrappers/FiniteStormSpoutWrapper.java | 111 ------- .../wrappers/StormFiniteSpoutWrapper.java | 186 ------------ .../wrappers/StormSpoutWrapper.java | 105 ------- .../wrappers/FiniteStormSpoutWrapperTest.java | 76 ----- .../wrappers/StormFiniteSpoutWrapperTest.java | 117 -------- .../wrappers/StormSpoutWrapperTest.java | 120 -------- .../README.md | 20 -- .../StormExclamationRemoteByClient.java | 83 ------ .../StormExclamationRemoteBySubmitter.java | 81 ------ .../singlejoin/SingleJoinTopology.java | 90 ------ .../singlejoin/StormSingleJoinLocal.java | 50 ---- .../singlejoin/stormoperators/AgeSpout.java | 54 ---- .../stormoperators/GenderSpout.java | 47 --- .../stormoperators/SingleJoinBolt.java | 132 --------- .../flink-storm-compatibility/pom.xml | 40 --- flink-contrib/flink-storm-examples/README.md | 20 ++ .../pom.xml | 66 +++-- .../storm/excamation/ExclamationLocal.java} | 18 +- .../excamation/ExclamationTopology.java | 49 ++-- .../excamation/ExclamationWithBolt.java} | 37 +-- .../excamation/ExclamationWithSpout.java} | 38 +-- .../operators}/ExclamationBolt.java | 2 +- .../flink/storm}/split/SpoutSplitExample.java | 26 +- .../storm/split/operators}/RandomSpout.java | 2 +- .../split/operators}/VerifyAndEnrichBolt.java | 2 +- .../flink/storm/util/AbstractBoltSink.java} | 6 +- .../flink/storm/util/AbstractLineSpout.java} | 8 +- .../flink/storm/util/BoltFileSink.java} | 8 +- .../flink/storm/util/BoltPrintSink.java} | 6 +- .../apache/flink/storm/util/FileSpout.java} | 10 +- .../flink/storm/util/FiniteFileSpout.java} | 14 +- .../storm/util/FiniteInMemorySpout.java} | 14 +- .../flink/storm/util/InMemorySpout.java} | 8 +- .../flink/storm}/util/OutputFormatter.java | 11 +- .../storm}/util/SimpleOutputFormatter.java | 11 +- .../storm}/util/TupleOutputFormatter.java | 2 +- .../wordcount/BoltTokenizerWordCount.java | 20 +- .../wordcount/BoltTokenizerWordCountPojo.java | 35 ++- .../BoltTokenizerWordCountWithNames.java | 31 +- .../wordcount/SpoutSourceWordCount.java | 26 +- .../storm/wordcount/WordCountLocal.java} | 13 +- .../wordcount/WordCountLocalByName.java} | 15 +- .../wordcount/WordCountRemoteByClient.java} | 15 +- .../WordCountRemoteBySubmitter.java} | 15 +- .../storm}/wordcount/WordCountTopology.java | 53 ++-- .../wordcount/operators/BoltCounter.java} | 14 +- .../operators/BoltCounterByName.java} | 14 +- .../wordcount/operators/BoltTokenizer.java} | 8 +- .../operators/BoltTokenizerByName.java} | 8 +- .../operators}/WordCountDataPojos.java | 2 +- .../operators}/WordCountDataTuple.java | 2 +- .../operators/WordCountFileSpout.java} | 10 +- .../operators/WordCountInMemorySpout.java} | 10 +- .../ExclamationWithBoltITCase.java} | 12 +- .../ExclamationWithSpoutITCase.java} | 12 +- .../StormExclamationLocalITCase.java | 10 +- .../exclamation/util/ExclamationData.java | 2 +- .../flink/storm}/split/BoltSplitITCase.java | 4 +- .../apache/flink/storm}/split/SplitBolt.java | 2 +- .../flink/storm}/split/SplitBoltTopology.java | 28 +- .../storm}/split/SplitSpoutTopology.java | 24 +- .../storm/split/SplitStreamBoltLocal.java} | 8 +- .../storm/split/SplitStreamSpoutLocal.java} | 8 +- .../flink/storm}/split/SpoutSplitITCase.java | 4 +- .../flink/storm/util}/StormTestBase.java | 4 +- .../BoltTokenizerWordCountITCase.java | 5 +- .../BoltTokenizerWordCountPojoITCase.java | 5 +- ...BoltTokenizerWordCountWithNamesITCase.java | 5 +- .../wordcount/SpoutSourceWordCountITCase.java | 5 +- .../wordcount/WordCountLocalITCase.java} | 9 +- .../wordcount/WordCountLocalNamedITCase.java} | 9 +- .../src/test/resources/log4j-test.properties | 0 .../src/test/resources/log4j.properties | 0 .../src/test/resources/logback-test.xml | 0 flink-contrib/flink-storm/README.md | 15 + .../pom.xml | 6 +- .../apache/flink/storm}/api/FlinkClient.java | 4 +- .../flink/storm}/api/FlinkLocalCluster.java | 5 +- .../storm/api}/FlinkOutputFieldsDeclarer.java | 10 +- .../flink/storm}/api/FlinkSubmitter.java | 2 +- .../flink/storm}/api/FlinkTopology.java | 2 +- .../storm}/api/FlinkTopologyBuilder.java | 46 ++- .../apache/flink/storm/util/FiniteSpout.java} | 13 +- .../flink/storm}/util/SplitStreamMapper.java | 2 +- .../flink/storm}/util/SplitStreamType.java | 6 +- .../util/SplitStreamTypeKeySelector.java | 4 +- .../apache/flink/storm}/util/StormConfig.java | 2 +- .../storm/util/StormStreamSelector.java} | 6 +- .../wrappers/AbstractStormCollector.java | 6 +- .../flink/storm/wrappers/BoltCollector.java} | 19 +- .../flink/storm/wrappers/BoltWrapper.java} | 50 ++-- .../storm/wrappers}/FlinkTopologyContext.java | 7 +- .../wrappers/SetupOutputFieldsDeclarer.java | 7 +- .../flink/storm/wrappers/SpoutCollector.java} | 11 +- .../flink/storm/wrappers/SpoutWrapper.java | 274 ++++++++++++++++++ .../flink/storm}/wrappers/StormTuple.java | 4 +- .../storm/wrappers/WrapperSetupHelper.java} | 18 +- .../api}/FlinkOutputFieldsDeclarerTest.java | 5 +- .../storm}/api/FlinkTopologyBuilderTest.java | 9 +- .../flink/storm}/api/FlinkTopologyTest.java | 3 +- .../org/apache/flink/storm}/api/TestBolt.java | 2 +- .../apache/flink/storm}/api/TestSpout.java | 2 +- .../flink/storm}/api/TestTopologyBuilder.java | 4 +- .../flink/storm}/util/AbstractTest.java | 2 +- .../flink/storm}/util/FiniteTestSpout.java | 2 +- .../storm/util/StormStreamSelectorTest.java} | 10 +- .../flink/storm}/util/TestDummyBolt.java | 5 +- .../flink/storm}/util/TestDummySpout.java | 5 +- .../apache/flink/storm}/util/TestSink.java | 2 +- .../storm/wrappers/BoltCollectorTest.java} | 34 +-- .../storm/wrappers/BoltWrapperTest.java} | 58 ++-- .../wrappers}/FlinkTopologyContextTest.java | 5 +- .../SetupOutputFieldsDeclarerTest.java | 5 +- .../storm/wrappers/SpoutCollectorTest.java} | 22 +- .../storm/wrappers/SpoutWrapperTest.java | 220 ++++++++++++++ .../flink/storm}/wrappers/StormTupleTest.java | 5 +- .../flink/storm}/wrappers/TestContext.java | 2 +- .../wrappers/WrapperSetupHelperTest.java} | 30 +- .../src/test/resources/log4j-test.properties | 0 .../src/test/resources/log4j.properties | 0 .../src/test/resources/logback-test.xml | 0 flink-contrib/pom.xml | 3 +- 126 files changed, 1192 insertions(+), 2202 deletions(-) delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/SplitStreamTypeKeySelector.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/README.md delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/SingleJoinTopology.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/StormSingleJoinLocal.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/AgeSpout.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/GenderSpout.java delete mode 100644 flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/SingleJoinBolt.java delete mode 100644 flink-contrib/flink-storm-compatibility/pom.xml create mode 100644 flink-contrib/flink-storm-examples/README.md rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples => flink-storm-examples}/pom.xml (80%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java => flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java} (77%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/excamation/ExclamationTopology.java (65%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java => flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java} (74%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java} (76%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/stormoperators => flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators}/ExclamationBolt.java (96%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/split/SpoutSplitExample.java (80%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators => flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators}/RandomSpout.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators => flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators}/VerifyAndEnrichBolt.java (96%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormBoltSink.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java} (93%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormSpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java} (86%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltFileSink.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java} (88%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltPrintSink.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java} (87%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormFileSpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java} (89%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java} (83%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java} (70%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormInMemorySpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java} (84%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/util/OutputFormatter.java (84%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/util/SimpleOutputFormatter.java (84%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/util/TupleOutputFormatter.java (96%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/wordcount/BoltTokenizerWordCount.java (84%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/wordcount/BoltTokenizerWordCountPojo.java (74%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/wordcount/BoltTokenizerWordCountWithNames.java (78%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/wordcount/SpoutSourceWordCount.java (84%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java} (87%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountNamedLocal.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java} (85%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java} (88%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java} (87%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/main/java/org/apache/flink/storm}/wordcount/WordCountTopology.java (62%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounterByName.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java} (80%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounter.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java} (80%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java} (89%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizerByName.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java} (89%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators}/WordCountDataPojos.java (95%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators}/WordCountDataTuple.java (94%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java} (77%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.java => flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java} (77%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java => flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java} (75%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java => flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.java} (75%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/exclamation/StormExclamationLocalITCase.java (79%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/exclamation/util/ExclamationData.java (99%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/split/BoltSplitITCase.java (88%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/split/SplitBolt.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/split/SplitBoltTopology.java (72%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/split/SplitSpoutTopology.java (75%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java => flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java} (87%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java => flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java} (87%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/split/SpoutSplitITCase.java (88%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api => flink-storm-examples/src/test/java/org/apache/flink/storm/util}/StormTestBase.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/wordcount/BoltTokenizerWordCountITCase.java (90%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/wordcount/BoltTokenizerWordCountPojoITCase.java (90%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/wordcount/BoltTokenizerWordCountWithNamesITCase.java (90%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility => flink-storm-examples/src/test/java/org/apache/flink/storm}/wordcount/SpoutSourceWordCountITCase.java (90%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java => flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.java} (82%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java => flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.java} (81%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core => flink-storm-examples}/src/test/resources/log4j-test.properties (100%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core => flink-storm-examples}/src/test/resources/log4j.properties (100%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core => flink-storm-examples}/src/test/resources/logback-test.xml (100%) create mode 100644 flink-contrib/flink-storm/README.md rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core => flink-storm}/pom.xml (95%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/api/FlinkClient.java (99%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/api/FlinkLocalCluster.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util => flink-storm/src/main/java/org/apache/flink/storm/api}/FlinkOutputFieldsDeclarer.java (92%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/api/FlinkSubmitter.java (99%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/api/FlinkTopology.java (98%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/api/FlinkTopologyBuilder.java (89%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java => flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java} (66%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/util/SplitStreamMapper.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/util/SplitStreamType.java (89%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/util/SplitStreamTypeKeySelector.java (92%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/util/StormConfig.java (94%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java => flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java} (88%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/wrappers/AbstractStormCollector.java (96%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java => flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java} (77%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java => flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java} (83%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util => flink-storm/src/main/java/org/apache/flink/storm/wrappers}/FlinkTopologyContext.java (95%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/wrappers/SetupOutputFieldsDeclarer.java (86%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java => flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java} (83%) create mode 100644 flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility => flink-storm/src/main/java/org/apache/flink/storm}/wrappers/StormTuple.java (98%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java => flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java} (94%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util => flink-storm/src/test/java/org/apache/flink/storm/api}/FlinkOutputFieldsDeclarerTest.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/api/FlinkTopologyBuilderTest.java (91%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/api/FlinkTopologyTest.java (95%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/api/TestBolt.java (96%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/api/TestSpout.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/api/TestTopologyBuilder.java (91%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/util/AbstractTest.java (96%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/util/FiniteTestSpout.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java => flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java} (82%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/util/TestDummyBolt.java (95%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/util/TestDummySpout.java (95%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/util/TestSink.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java => flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java} (69%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java => flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java} (85%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util => flink-storm/src/test/java/org/apache/flink/storm/wrappers}/FlinkTopologyContextTest.java (97%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/wrappers/SetupOutputFieldsDeclarerTest.java (95%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java => flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java} (78%) create mode 100644 flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/wrappers/StormTupleTest.java (99%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility => flink-storm/src/test/java/org/apache/flink/storm}/wrappers/TestContext.java (96%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java => flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java} (92%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples => flink-storm}/src/test/resources/log4j-test.properties (100%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples => flink-storm}/src/test/resources/log4j.properties (100%) rename flink-contrib/{flink-storm-compatibility/flink-storm-compatibility-examples => flink-storm}/src/test/resources/logback-test.xml (100%) diff --git a/docs/apis/storm_compatibility.md b/docs/apis/storm_compatibility.md index d676db889c6fa..bf80d4e908d38 100644 --- a/docs/apis/storm_compatibility.md +++ b/docs/apis/storm_compatibility.md @@ -36,33 +36,34 @@ This document shows how to use existing Storm code with Flink. # Project Configuration -Support for Storm is contained in the `flink-storm-compatibility-core` Maven module. -The code resides in the `org.apache.flink.stormcompatibility` package. +Support for Storm is contained in the `flink-storm` Maven module. +The code resides in the `org.apache.flink.storm` package. Add the following dependency to your `pom.xml` if you want to execute Storm code in Flink. ~~~xml org.apache.flink - flink-storm-compatibility-core + flink-storm {{site.version}} ~~~ -**Please note**: `flink-storm-compatibility-core` is not part of the provided binary Flink distribution. -Thus, you need to include `flink-storm-compatiblitly-core` classes (and their dependencies) in your program jar that is submitted to Flink's JobManager. -See *WordCount Storm* within `flink-storm-compatibility-example/pom.xml` for an example how to package a jar correctly. +**Please note**: `flink-storm` is not part of the provided binary Flink distribution. +Thus, you need to include `flink-storm` classes (and their dependencies) in your program jar that is submitted to Flink's JobManager. +See *WordCount Storm* within `flink-storm-examples/pom.xml` for an example how to package a jar correctly. # Execute Storm Topologies -Flink provides a Storm compatible API (`org.apache.flink.stormcompatibility.api`) that offers replacements for the following classes: +Flink provides a Storm compatible API (`org.apache.flink.storm.api`) that offers replacements for the following classes: - `TopologyBuilder` replaced by `FlinkTopologyBuilder` - `StormSubmitter` replaced by `FlinkSubmitter` - `NimbusClient` and `Client` replaced by `FlinkClient` - `LocalCluster` replaced by `FlinkLocalCluster` -In order to submit a Storm topology to Flink, it is sufficient to replace the used Storm classed with their Flink replacements in the original Storm client code that assembles the topology. +In order to submit a Storm topology to Flink, it is sufficient to replace the used Storm classes with their Flink replacements in the Storm *client code that assembles* the topology. +The actual runtime code, ie, Spouts and Bolts, can be uses *unmodified*. If a topology is executed in a remote cluster, parameters `nimbus.host` and `nimbus.thrift.port` are used as `jobmanger.rpc.address` and `jobmanger.rpc.port`, respectively. If a parameter is not specified, the value is taken from `flink-conf.yaml`. @@ -71,10 +72,11 @@ If a parameter is not specified, the value is taken from `flink-conf.yaml`. ~~~java FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); // replaces: TopologyBuilder builder = new FlinkTopology(); -builder.setSpout("source", new StormFileSpout(inputFilePath)); -builder.setBolt("tokenizer", new StormBoltTokenizer()).shuffleGrouping("source"); -builder.setBolt("counter", new StormBoltCounter()).fieldsGrouping("tokenizer", new Fields("word")); -builder.setBolt("sink", new StormBoltFileSink(outputFilePath)).shuffleGrouping("counter"); +// actual topology assembling code and used Spouts/Bolts can be used as-is +builder.setSpout("source", new FileSpout(inputFilePath)); +builder.setBolt("tokenizer", new BoltTokenizer()).shuffleGrouping("source"); +builder.setBolt("counter", new BoltCounter()).fieldsGrouping("tokenizer", new Fields("word")); +builder.setBolt("sink", new BoltFileSink(outputFilePath)).shuffleGrouping("counter"); Config conf = new Config(); if(runLocal) { // submit to test cluster @@ -93,7 +95,7 @@ if(runLocal) { // submit to test cluster # Embed Storm Operators in Flink Streaming Programs As an alternative, Spouts and Bolts can be embedded into regular streaming programs. -The Storm compatibility layer offers a wrapper classes for each, namely `StormSpoutWrapper` and `StormBoltWrapper` (`org.apache.flink.stormcompatibility.wrappers`). +The Storm compatibility layer offers a wrapper classes for each, namely `SpoutWrapper` and `BoltWrapper` (`org.apache.flink.storm.wrappers`). Per default, both wrappers convert Storm output tuples to Flink's [Tuple](programming_guide.html#tuples-and-case-classes) types (ie, `Tuple0` to `Tuple25` according to the number of fields of the Storm tuples). For single field output tuples a conversion to the field's data type is also possible (eg, `String` instead of `Tuple1`). @@ -104,7 +106,7 @@ In order to get the correct `TypeInformation` object, Flink's `TypeExtractor` ca ## Embed Spouts In order to use a Spout as Flink source, use `StreamExecutionEnvironment.addSource(SourceFunction, TypeInformation)`. -The Spout object is handed to the constructor of `StormSpoutWrapper` that serves as first argument to `addSource(...)`. +The Spout object is handed to the constructor of `SpoutWrapper` that serves as first argument to `addSource(...)`. The generic type declaration `OUT` specifies the type of the source output stream.
@@ -114,7 +116,7 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm // stream has `raw` type (single field output streams only) DataStream rawInput = env.addSource( - new StormSpoutWrapper(new StormFileSpout(localFilePath), new String[] { Utils.DEFAULT_STREAM_ID }), // emit default output stream as raw type + new SpoutWrapper(new FileSpout(localFilePath), new String[] { Utils.DEFAULT_STREAM_ID }), // emit default output stream as raw type TypeExtractor.getForClass(String.class)); // output type // process data stream @@ -123,15 +125,15 @@ DataStream rawInput = env.addSource(
-If a Spout emits a finite number of tuples, `StormFiniteSpoutWrapper` can be used instead of `StormSpoutWrapper`. -Using `StormFiniteSpoutWrapper` allows the Flink program to shut down automatically after all data is processed. -If `StormSpoutWrapper` is used, the program will run until it is [canceled](cli.html) manually. +If a Spout emits a finite number of tuples, `SpoutWrapper` can be configures to terminate automatically by setting `numberOfInvocations` parameter in its constructor. +This allows the Flink program to shut down automatically after all data is processed. +Per default the program will run until it is [canceled](cli.html) manually. ## Embed Bolts In order to use a Bolt as Flink operator, use `DataStream.transform(String, TypeInformation, OneInputStreamOperator)`. -The Bolt object is handed to the constructor of `StormBoltWrapper` that serves as last argument to `transform(...)`. +The Bolt object is handed to the constructor of `BoltWrapper` that serves as last argument to `transform(...)`. The generic type declarations `IN` and `OUT` specify the type of the operator's input and output stream, respectively.
@@ -143,7 +145,7 @@ DataStream text = env.readTextFile(localFilePath); DataStream> counts = text.transform( "tokenizer", // operator name TypeExtractor.getForObject(new Tuple2("", 0)), // output type - new StormBoltWrapper>(new StormBoltTokenizer())); // Bolt operator + new BoltWrapper>(new BoltTokenizer())); // Bolt operator // do further processing [...] @@ -164,16 +166,16 @@ For this case, Flink expects either a corresponding public member variable or pu For example, if a Bolt accesses a field via name `sentence` (eg, `String s = input.getStringByField("sentence");`), the input POJO class must have a member variable `public String sentence;` or method `public String getSentence() { ... };` (pay attention to camel-case naming). For `Tuple` input types, it is required to specify the input schema using Storm's `Fields` class. -For this case, the constructor of `StormBoltWrapper` takes an additional argument: `new StormBoltWrapper, Tuple2>(new StormBoltTokenizerByName(), new Fields("sentence"))`. +For this case, the constructor of `BoltWrapper` takes an additional argument: `new BoltWrapper, ...>(..., new Fields("sentence"))`. The input type is `Tuple1` and `Fields("sentence")` specify that `input.getStringByField("sentence")` is equivalent to `input.getString(0)`. -See [BoltTokenizerWordCountPojo](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java) and [BoltTokenizerWordCountWithNames](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java) for examples. +See [BoltTokenizerWordCountPojo](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java) and [BoltTokenizerWordCountWithNames](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java) for examples. ## Configuring Spouts and Bolts In Storm, Spouts and Bolts can be configured with a globally distributed `Map` object that is given to `submitTopology(...)` method of `LocalCluster` or `StormSubmitter`. This `Map` is provided by the user next to the topology and gets forwarded as a parameter to the calls `Spout.open(...)` and `Bolt.prepare(...)`. -If a whole topology is executed using `FlinkTopologyBuilder` etc., there is no special attention required – it works as in regular Storm. +If a whole topology is executed in Flink using `FlinkTopologyBuilder` etc., there is no special attention required – it works as in regular Storm. For embedded usage, Flink's configuration mechanism must be used. A global configuration can be set in a `StreamExecutionEnvironment` via `.getConfig().setGlobalJobParameters(...)`. @@ -202,12 +204,12 @@ env.getConfig().setGlobalJobParameters(config); ## Multiple Output Streams Flink can also handle the declaration of multiple output streams for Spouts and Bolts. -If a whole topology is executed using `FlinkTopologyBuilder` etc., there is no special attention required – it works as in regular Storm. +If a whole topology is executed in Flink using `FlinkTopologyBuilder` etc., there is no special attention required – it works as in regular Storm. -For embedded usage, the output stream will be of data type `SplitStreamType` and must be split by using `DataStream.split(...)` and `SplitDataStream.select(...)`. -Flink provides the predefined output selector `FlinkStormStreamSelector` for `.split(...)` already. +For embedded usage, the output stream will be of data type `SplitStreamType` and must be split by using `DataStream.split(...)` and `SplitStream.select(...)`. +Flink provides the predefined output selector `StormStreamSelector` for `.split(...)` already. Furthermore, the wrapper type `SplitStreamTuple` can be removed using `SplitStreamMapper`. -If a data stream of type `SplitStreamTuple` is used as input for a Bolt, `SplitStreamTuple` must **not** be removed – `StormBoltWrapper` removes it automatically. +If a data stream of type `SplitStreamTuple` is used as input for a Bolt, it is **not** required to strip the wrapper – `BoltWrapper` removes it automatically.
@@ -217,11 +219,11 @@ If a data stream of type `SplitStreamTuple` is used as input for a Bolt, `Spl // get DataStream from Spout or Bolt which declares two output streams s1 and s2 with output type SomeType DataStream> multiStream = ... -SplitDataStream> splitStream = multiStream.split(new FlinkStormStreamSelector()); +SplitStream> splitStream = multiStream.split(new StormStreamSelector()); // remove SplitStreamMapper to get data stream of type SomeType DataStream s1 = splitStream.select("s1").map(new SplitStreamMapper).returns(SomeType.classs); -// apply Bolt directly, without stripping SplitStreamMapper +// apply Bolt directly, without stripping SplitStreamType DataStream s2 = splitStream.select("s2").transform(/* use Bolt for further processing */); // do further processing on s1 and s2 @@ -230,67 +232,48 @@ DataStream s2 = splitStream.select("s2").transform(/* use Bolt f
-See [SpoutSplitExample.java](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java) for a full example. +See [SpoutSplitExample.java](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java) for a full example. # Flink Extensions -## Finite Storm Spouts +## Finite Spouts -In Flink streaming, sources can be finite – i.e., emit a finite number of records and stop after emitting the last record –, however, Storm spouts always emit infinite streams. -The bridge between the two approach is the `FiniteStormSpout` interface which, in addition to `IRichSpout`, contains a `reachedEnd()` method, where the user can specify a stopping-condition. -The user can create a finite Storm spout by implementing this interface instead of `IRichSpout`, and implementing the `reachedEnd()`method in addition. -When used as part of a Flink topology, a `FiniteStormSpout` should be wrapped by `FiniteStormSpoutWrapper`. +In Flink, streaming sources can be finite, ie, emit a finite number of records and stop after emitting the last record. However, Spouts usually emit infinite streams. +The bridge between the two approaches is the `FiniteSpout` interface which, in addition to `IRichSpout`, contains a `reachedEnd()` method, where the user can specify a stopping-condition. +The user can create a finite Spout by implementing this interface instead of (or additionally to) `IRichSpout`, and implementing the `reachedEnd()` method in addition. +In contrast to a `SpoutWrapper` that is configured to emit a finite number of tuples, `FiniteSpout` interface allows to implement more complex termination criteria. -Although finite Storm spouts are not necessary to embed Storm spouts into a Flink streaming program or to submit a whole Storm topology to Flink, there are cases where they may come in handy: +Although finite Spouts are not necessary to embed Spouts into a Flink streaming program or to submit a whole Storm topology to Flink, there are cases where they may come in handy: - * to achieve that a native Storm spout behaves the same way as a finite Flink source with minimal modifications - * the user wants to process a stream only for some time; after that, the spout can stop automatically + * to achieve that a native Spout behaves the same way as a finite Flink source with minimal modifications + * the user wants to process a stream only for some time; after that, the Spout can stop automatically * reading a file into a stream * for testing purposes -A `FiniteStormSpout` can be still used as a normal, infinite Storm spout by changing its wrapper class to `StormSpoutWraper` in the Flink topology. - -An example of a finite Storm spout that emits records for 10 seconds only: +An example of a finite Spout that emits records for 10 seconds only:
~~~java -public class TimedFiniteStormSpout extends AbstractStormSpout implements FiniteStormSpout { - [...] +public class TimedFiniteSpout extends BaseRichSpout implements FiniteSpout { + [...] // implemente open(), nextTuple(), ... + private long starttime = System.currentTimeMillis(); public boolean reachedEnd() { return System.currentTimeMillis() - starttime > 10000l; } - [...] } ~~~
-Using a `FiniteStormSpout` in a Flink topology: - -
-
-~~~java -StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - -DataStream rawInput = env.addSource( - new FiniteStormSpoutWrapper(new TimedFiniteStormSpout(), true) - TypeExtractor.getForClass(String.class)); - -// process data stream -[...] -~~~ -
-
- # Storm Compatibility Examples -You can find more examples in Maven module `flink-storm-compatibilty-examples`. -For the different versions of WordCount, see [README.md](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/README.md). +You can find more examples in Maven module `flink-storm-examples`. +For the different versions of WordCount, see [README.md](https://github.com/apache/flink/tree/master/flink-contrib/flink-storm-examples/README.md). To run the examples, you need to assemble a correct jar file. -`flink-storm-compatibility-examples-0.10-SNAPSHOT.jar` is **no** valid jar file for job execution (it is only a standard maven artifact). +`flink-storm-examples-0.10-SNAPSHOT.jar` is **no** valid jar file for job execution (it is only a standard maven artifact). There are example jars for embedded Spout and Bolt, namely `WordCount-SpoutSource.jar` and `WordCount-BoltTokenizer.jar`, respectively. Compare `pom.xml` to see how both jars are built. diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md deleted file mode 100644 index 9663fc714f9f3..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md +++ /dev/null @@ -1,12 +0,0 @@ -# flink-storm-compatibility - -The Storm compatibility layer allows to embed spouts or bolt unmodified within a regular Flink streaming program (`StormSpoutWrapper` and `StormBoltWrapper`). Additionally, a whole Storm topology can be submitted to Flink (see `FlinkTopologyBuilder`, `FlinkLocalCluster`, and `FlinkSubmitter`). Only a few minor changes to the original submitting code are required. The code that builds the topology itself, can be reused unmodified. See `flink-storm-examples` for a simple word-count example. - -The following Strom features are not (yet/fully) supported by the compatibility layer right now: -* tuple meta information -* no fault-tolerance guarantees (ie, calls to `ack()`/`fail()` and anchoring is ignored) -* for whole Storm topologies the following is not supported by Flink: - * direct emit connection pattern - * activating/deactivating and rebalancing of topologies - * task hooks - * custom metrics diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/SplitStreamTypeKeySelector.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/SplitStreamTypeKeySelector.java deleted file mode 100644 index 30227b8a6d932..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/SplitStreamTypeKeySelector.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.stormcompatibility.api; - -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.stormcompatibility.util.SplitStreamType; -import org.apache.flink.streaming.util.keys.KeySelectorUtil; -import org.apache.flink.streaming.util.keys.KeySelectorUtil.ArrayKeySelector; - -/** - * {@link SplitStreamTypeKeySelector} is a specific grouping key selector for streams that are selected via - * {@link FlinkStormStreamSelector} from a Spout or Bolt that declares multiple output streams. - * - * It extracts the wrapped {@link Tuple} type from the {@link SplitStreamType} tuples and applies a regular - * {@link ArrayKeySelector} on it. - */ -public class SplitStreamTypeKeySelector implements KeySelector, Tuple> { - private static final long serialVersionUID = 4672434660037669254L; - - private final ArrayKeySelector selector; - - public SplitStreamTypeKeySelector(int... fields) { - this.selector = new KeySelectorUtil.ArrayKeySelector(fields); - } - - @Override - public Tuple getKey(SplitStreamType value) throws Exception { - return selector.getKey(value.value); - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java deleted file mode 100644 index ccd29bb112caf..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * 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.stormcompatibility.wrappers; - -import java.util.Collection; -import java.util.HashMap; - -import backtype.storm.generated.StormTopology; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.topology.IRichSpout; - -import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; -import org.apache.flink.api.java.tuple.Tuple0; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple25; -import org.apache.flink.stormcompatibility.util.StormConfig; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; - -/** - * A {@link AbstractStormSpoutWrapper} wraps an {@link IRichSpout} in order to execute the Storm bolt within a Flink - * Streaming program. It takes the spout's output tuples and transforms them into Flink tuples of type {@code OUT} (see - * {@link StormSpoutCollector} for supported types).
- *
- * CAUTION: currently, only simple spouts are supported! (ie, spouts that do not use the Storm configuration - * Map or TopologyContext that is provided by the spouts's prepare(..) method. - * Furthermore, ack and fail back calls as well as tuple IDs are not supported so far. - */ -public abstract class AbstractStormSpoutWrapper extends RichParallelSourceFunction { - private static final long serialVersionUID = 4993283609095408765L; - - /** Number of attributes of the bolt's output tuples per stream. */ - private final HashMap numberOfAttributes; - /** The wrapped Storm {@link IRichSpout spout}. */ - protected final IRichSpout spout; - /** The wrapper of the given Flink collector. */ - protected StormSpoutCollector collector; - /** Indicates, if the source is still running or was canceled. */ - protected volatile boolean isRunning = true; - /** The original Storm topology. */ - protected StormTopology stormTopology; - - /** - * Instantiates a new {@link AbstractStormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such - * that it can be used within a Flink streaming program. The output type will be one of {@link Tuple0} to - * {@link Tuple25} depending on the spout's declared number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [0;25]. - */ - public AbstractStormSpoutWrapper(final IRichSpout spout) throws IllegalArgumentException { - this(spout, null); - } - - /** - * Instantiates a new {@link AbstractStormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such - * that it can be used within a Flink streaming program. The output type can be any type if parameter - * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is - * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared - * number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public AbstractStormSpoutWrapper(final IRichSpout spout, - final Collection rawOutputs) - throws IllegalArgumentException { - this.spout = spout; - this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(spout, rawOutputs); - } - - /** - * Sets the original Storm topology. - * - * @param stormTopology - * The original Storm topology. - */ - public void setStormTopology(StormTopology stormTopology) { - this.stormTopology = stormTopology; - } - - @Override - public final void run(final SourceContext ctx) throws Exception { - this.collector = new StormSpoutCollector(this.numberOfAttributes, ctx); - - GlobalJobParameters config = super.getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); - StormConfig stormConfig = new StormConfig(); - - if (config != null) { - if (config instanceof StormConfig) { - stormConfig = (StormConfig) config; - } else { - stormConfig.putAll(config.toMap()); - } - } - - this.spout.open(stormConfig, - StormWrapperSetupHelper.createTopologyContext( - (StreamingRuntimeContext) super.getRuntimeContext(), - this.spout, - this.stormTopology, - null), - new SpoutOutputCollector(this.collector)); - this.spout.activate(); - this.execute(); - } - - /** - * Needs to be implemented to call the given Spout's {@link IRichSpout#nextTuple() nextTuple()} method. This method - * might use a {@code while(true)}-loop to emit an infinite number of tuples. - */ - protected abstract void execute(); - - /** - * {@inheritDoc} - *

- * Sets the {@link #isRunning} flag to {@code false}. - */ - @Override - public void cancel() { - this.isRunning = false; - } - - @Override - public void close() throws Exception { - this.spout.close(); - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java deleted file mode 100644 index f499ecc167708..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.stormcompatibility.wrappers; - -import java.util.Collection; - -import org.apache.flink.api.java.tuple.Tuple0; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple25; -import org.apache.flink.stormcompatibility.util.FiniteStormSpout; - -import com.google.common.collect.Sets; - -/** - * A {@link FiniteStormSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls the wrapped - * {@link FiniteStormSpout}'s {@link FiniteStormSpout#nextTuple()} method until {@link - * FiniteStormSpout#reachedEnd()} is true. - */ -public class FiniteStormSpoutWrapper extends AbstractStormSpoutWrapper { - private static final long serialVersionUID = -218340336648247605L; - - private FiniteStormSpout finiteSpout; - - /** - * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link FiniteStormSpout spout} such - * that it can be used within a Flink streaming program. The output type will be one of {@link Tuple0} to - * {@link Tuple25} depending on the spout's declared number of attributes. - * - * @param spout - * The Storm {@link FiniteStormSpout spout} to be used. - * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [0;25]. - */ - public FiniteStormSpoutWrapper(FiniteStormSpout spout) - throws IllegalArgumentException { - super(spout); - this.finiteSpout = spout; - } - - /** - * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link FiniteStormSpout spout} such - * that it can be used within a Flink streaming program. The output type can be any type if parameter - * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is - * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared - * number of attributes. - * - * @param spout - * The Storm {@link FiniteStormSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public FiniteStormSpoutWrapper(final FiniteStormSpout spout, final String[] rawOutputs) - throws IllegalArgumentException { - this(spout, Sets.newHashSet(rawOutputs)); - } - - /** - * Instantiates a new {@link FiniteStormSpoutWrapper} that wraps the given Storm {@link FiniteStormSpout spout} such - * that it can be used within a Flink streaming program. The output type can be any type if parameter - * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is - * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared - * number of attributes. - * - * @param spout - * The Storm {@link FiniteStormSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public FiniteStormSpoutWrapper(final FiniteStormSpout spout, final Collection rawOutputs) - throws IllegalArgumentException { - super(spout, rawOutputs); - this.finiteSpout = spout; - } - - /** - * Calls the {@link FiniteStormSpout#nextTuple()} method until {@link FiniteStormSpout#reachedEnd()} is true or - * {@link FiniteStormSpout#cancel()} is called. - */ - @Override - protected void execute() { - while (super.isRunning && !finiteSpout.reachedEnd()) { - finiteSpout.nextTuple(); - } - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java deleted file mode 100644 index 45eb56c71fc19..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * 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.stormcompatibility.wrappers; - -import java.util.Collection; - -import backtype.storm.topology.IRichSpout; - -import org.apache.flink.api.java.tuple.Tuple0; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple25; - -import com.google.common.collect.Sets; - -/** - * A {@link StormFiniteSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls {@link IRichSpout#nextTuple() - * nextTuple()} for finite number of times before - * {@link #run(org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext)} returns. The number of - * {@code nextTuple()} calls can be specified as a certain number of invocations or can be undefined. In the undefined - * case, the {@code run(...)} method return if no record was emitted to the output collector for the first time. - */ -public class StormFiniteSpoutWrapper extends AbstractStormSpoutWrapper { - private static final long serialVersionUID = 3883246587044801286L; - - /** The number of {@link IRichSpout#nextTuple()} calls */ - private int numberOfInvocations; - - /** - * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} - * method of the given Storm {@link IRichSpout spout} as long as it emits records to the output collector. The - * output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of - * attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [0;25]. - */ - public StormFiniteSpoutWrapper(final IRichSpout spout) throws IllegalArgumentException { - this(spout, (Collection) null, -1); - } - - /** - * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} - * method of the given Storm {@link IRichSpout spout} {@code numberOfInvocations} times. The output type will be one - * of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param numberOfInvocations - * The number of calls to {@link IRichSpout#nextTuple()}. - * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [0;25]. - */ - public StormFiniteSpoutWrapper(final IRichSpout spout, final int numberOfInvocations) - throws IllegalArgumentException { - this(spout, (Collection) null, numberOfInvocations); - } - - /** - * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} - * method of the given Storm {@link IRichSpout spout} as long as it emits records to the output collector. The - * output type can be any type if parameter {@code rawOutput} is {@code true} and the spout's number of declared - * output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to - * {@link Tuple25} depending on the spout's declared number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public StormFiniteSpoutWrapper(final IRichSpout spout, final String[] rawOutputs) - throws IllegalArgumentException { - this(spout, Sets.newHashSet(rawOutputs), -1); - } - - /** - * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} - * method of the given Storm {@link IRichSpout spout} as long as it emits records to the output collector. The - * output type can be any type if parameter {@code rawOutput} is {@code true} and the spout's number of declared - * output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to - * {@link Tuple25} depending on the spout's declared number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public StormFiniteSpoutWrapper(final IRichSpout spout, final Collection rawOutputs) - throws IllegalArgumentException { - this(spout, rawOutputs, -1); - } - - /** - * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} - * method of the given Storm {@link IRichSpout spout} {@code numberOfInvocations} times. The output type can be any - * type if parameter {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If - * {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on - * the spout's declared number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. - * @param numberOfInvocations - * The number of calls to {@link IRichSpout#nextTuple()}. - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public StormFiniteSpoutWrapper(final IRichSpout spout, final String[] rawOutputs, - final int numberOfInvocations) throws IllegalArgumentException { - super(spout, Sets.newHashSet(rawOutputs)); - this.numberOfInvocations = numberOfInvocations; - } - - /** - * Instantiates a new {@link StormFiniteSpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} - * method of the given Storm {@link IRichSpout spout} {@code numberOfInvocations} times. The output type can be any - * type if parameter {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If - * {@code rawOutput} is {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on - * the spout's declared number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. - * @param numberOfInvocations - * The number of calls to {@link IRichSpout#nextTuple()}. - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public StormFiniteSpoutWrapper(final IRichSpout spout, final Collection rawOutputs, - final int numberOfInvocations) throws IllegalArgumentException { - super(spout, rawOutputs); - this.numberOfInvocations = numberOfInvocations; - } - - /** - * Calls {@link IRichSpout#nextTuple()} for the given number of times. - */ - @Override - protected void execute() { - if (this.numberOfInvocations >= 0) { - while ((--this.numberOfInvocations >= 0) && super.isRunning) { - super.spout.nextTuple(); - } - } else { - do { - super.collector.tupleEmitted = false; - super.spout.nextTuple(); - } while (super.collector.tupleEmitted && super.isRunning); - } - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java deleted file mode 100644 index 300b241312c55..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.stormcompatibility.wrappers; - -import java.util.Collection; - -import backtype.storm.topology.IRichSpout; - -import org.apache.flink.api.java.tuple.Tuple0; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.api.java.tuple.Tuple25; - -import com.google.common.collect.Sets; - -/** - * A {@link StormSpoutWrapper} is an {@link AbstractStormSpoutWrapper} that calls the wrapped spout's - * {@link IRichSpout#nextTuple() nextTuple()} method in in infinite loop. - */ -public class StormSpoutWrapper extends AbstractStormSpoutWrapper { - private static final long serialVersionUID = -218340336648247605L; - - /** - * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it can - * be used within a Flink streaming program. The output type will be one of {@link Tuple0} to {@link Tuple25} - * depending on the spout's declared number of attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @throws IllegalArgumentException - * If the number of declared output attributes is not with range [0;25]. - */ - public StormSpoutWrapper(final IRichSpout spout) throws IllegalArgumentException { - super(spout, null); - } - - /** - * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it can - * be used within a Flink streaming program. The output type can be any type if parameter {@code rawOutput} is - * {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the - * output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of - * attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. (Can be {@code null}.) - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public StormSpoutWrapper(final IRichSpout spout, final String[] rawOutputs) - throws IllegalArgumentException { - super(spout, Sets.newHashSet(rawOutputs)); - } - - /** - * Instantiates a new {@link StormSpoutWrapper} that wraps the given Storm {@link IRichSpout spout} such that it can - * be used within a Flink streaming program. The output type can be any type if parameter {@code rawOutput} is - * {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the - * output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of - * attributes. - * - * @param spout - * The Storm {@link IRichSpout spout} to be used. - * @param rawOutputs - * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be - * of a raw type. (Can be {@code null}.) - * @throws IllegalArgumentException - * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if - * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range - * [0;25]. - */ - public StormSpoutWrapper(final IRichSpout spout, final Collection rawOutputs) - throws IllegalArgumentException { - super(spout, rawOutputs); - } - - /** - * Calls {@link IRichSpout#nextTuple()} in an infinite loop until {@link #cancel()} is called. - */ - @Override - protected void execute() { - while (super.isRunning) { - super.spout.nextTuple(); - } - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java deleted file mode 100644 index b44e8a1292375..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.stormcompatibility.wrappers; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.stormcompatibility.util.FiniteStormSpout; -import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; -import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@RunWith(PowerMockRunner.class) -@PrepareForTest(StormWrapperSetupHelper.class) -public class FiniteStormSpoutWrapperTest { - - @SuppressWarnings("unchecked") - @Test - public void runAndExecuteTest1() throws Exception { - final FiniteStormSpout stormSpout = mock(FiniteStormSpout.class); - when(stormSpout.reachedEnd()).thenReturn(false, false, false, true, false, false, true); - - final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); - when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); - when(taskContext.getTaskName()).thenReturn("name"); - - final FiniteStormSpoutWrapper wrapper = new FiniteStormSpoutWrapper(stormSpout); - wrapper.setRuntimeContext(taskContext); - - wrapper.run(mock(SourceContext.class)); - verify(stormSpout, times(3)).nextTuple(); - } - - @SuppressWarnings("unchecked") - @Test - public void runAndExecuteTest2() throws Exception { - final FiniteStormSpout stormSpout = mock(FiniteStormSpout.class); - when(stormSpout.reachedEnd()).thenReturn(true, false, true, false, true, false, true); - - final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); - when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); - when(taskContext.getTaskName()).thenReturn("name"); - - final FiniteStormSpoutWrapper wrapper = new FiniteStormSpoutWrapper(stormSpout); - wrapper.setRuntimeContext(taskContext); - - wrapper.run(mock(SourceContext.class)); - verify(stormSpout, never()).nextTuple(); - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java deleted file mode 100644 index 77f1b05954a94..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.stormcompatibility.wrappers; - -import backtype.storm.topology.IRichSpout; -import backtype.storm.tuple.Fields; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.stormcompatibility.util.AbstractTest; -import org.apache.flink.stormcompatibility.util.FiniteTestSpout; -import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; -import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import java.util.LinkedList; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@RunWith(PowerMockRunner.class) -@PrepareForTest(StormWrapperSetupHelper.class) -public class StormFiniteSpoutWrapperTest extends AbstractTest { - - @SuppressWarnings("unchecked") - @Test - public void testRunExecuteFixedNumber() throws Exception { - final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); - declarer.declare(new Fields("dummy")); - PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - - final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); - when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); - when(taskContext.getTaskName()).thenReturn("name"); - - final IRichSpout spout = mock(IRichSpout.class); - final int numberOfCalls = this.r.nextInt(50); - final StormFiniteSpoutWrapper spoutWrapper = new StormFiniteSpoutWrapper(spout, numberOfCalls); - spoutWrapper.setRuntimeContext(taskContext); - - spoutWrapper.run(mock(SourceContext.class)); - verify(spout, times(numberOfCalls)).nextTuple(); - } - - @Test - public void testRunExecute() throws Exception { - final int numberOfCalls = this.r.nextInt(50); - - final LinkedList> expectedResult = new LinkedList>(); - for (int i = numberOfCalls - 1; i >= 0; --i) { - expectedResult.add(new Tuple1(new Integer(i))); - } - - final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); - when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); - when(taskContext.getTaskName()).thenReturn("name"); - - final IRichSpout spout = new FiniteTestSpout(numberOfCalls); - final StormFiniteSpoutWrapper> spoutWrapper = new StormFiniteSpoutWrapper>( - spout); - spoutWrapper.setRuntimeContext(taskContext); - - final TestContext collector = new TestContext(); - spoutWrapper.run(collector); - - Assert.assertEquals(expectedResult, collector.result); - } - - @Test - public void testCancel() throws Exception { - final int numberOfCalls = 5 + this.r.nextInt(5); - - final LinkedList> expectedResult = new LinkedList>(); - expectedResult.add(new Tuple1(new Integer(numberOfCalls - 1))); - - StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); - when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); - when(taskContext.getTaskName()).thenReturn("name"); - - final IRichSpout spout = new FiniteTestSpout(numberOfCalls); - final StormFiniteSpoutWrapper> spoutWrapper = new StormFiniteSpoutWrapper>( - spout); - - spoutWrapper.cancel(); - final TestContext collector = new TestContext(); - spoutWrapper.run(collector); - - Assert.assertEquals(expectedResult, collector.result); - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java deleted file mode 100644 index f4fb4daacf60d..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.stormcompatibility.wrappers; - -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.stormcompatibility.util.AbstractTest; -import org.apache.flink.stormcompatibility.util.FiniteTestSpout; -import org.apache.flink.stormcompatibility.util.StormConfig; -import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; -import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import java.util.LinkedList; -import java.util.Map; - -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Matchers.same; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@RunWith(PowerMockRunner.class) -@PrepareForTest(StormWrapperSetupHelper.class) -public class StormSpoutWrapperTest extends AbstractTest { - - @SuppressWarnings({ "rawtypes", "unchecked" }) - @Test - public void testRunPrepare() throws Exception { - final StormConfig stormConfig = new StormConfig(); - final Configuration flinkConfig = new Configuration(); - - final ExecutionConfig taskConfig = mock(ExecutionConfig.class); - when(taskConfig.getGlobalJobParameters()).thenReturn(null).thenReturn(stormConfig) - .thenReturn(flinkConfig); - - final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); - when(taskContext.getExecutionConfig()).thenReturn(taskConfig); - - final IRichSpout spout = mock(IRichSpout.class); - final StormSpoutWrapper spoutWrapper = new StormSpoutWrapper(spout); - spoutWrapper.setRuntimeContext(taskContext); - spoutWrapper.isRunning = false; - - // test without configuration - spoutWrapper.run(mock(SourceContext.class)); - verify(spout).open(any(Map.class), any(TopologyContext.class), - any(SpoutOutputCollector.class)); - - // test with StormConfig - spoutWrapper.run(mock(SourceContext.class)); - verify(spout).open(same(stormConfig), any(TopologyContext.class), - any(SpoutOutputCollector.class)); - - // test with Configuration - spoutWrapper.run(mock(SourceContext.class)); - verify(spout, times(3)).open(eq(flinkConfig.toMap()), any(TopologyContext.class), - any(SpoutOutputCollector.class)); - } - - @Test - public void testRunExecuteCancelInfinite() throws Exception { - final int numberOfCalls = 5 + this.r.nextInt(5); - - final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); - when(taskContext.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); - when(taskContext.getTaskName()).thenReturn("name"); - - final IRichSpout spout = new FiniteTestSpout(numberOfCalls); - - - final StormSpoutWrapper> spoutWrapper = new StormSpoutWrapper>(spout); - spoutWrapper.setRuntimeContext(taskContext); - - spoutWrapper.cancel(); - final TestContext collector = new TestContext(); - spoutWrapper.run(collector); - - Assert.assertEquals(new LinkedList>(), collector.result); - } - - @Test - public void testClose() throws Exception { - final IRichSpout spout = mock(IRichSpout.class); - final StormSpoutWrapper> spoutWrapper = new StormSpoutWrapper>(spout); - - spoutWrapper.close(); - - verify(spout).close(); - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/README.md b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/README.md deleted file mode 100644 index 6290df2c38c4c..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/README.md +++ /dev/null @@ -1,20 +0,0 @@ -# flink-storm-examples - -This module contains multiple versions of a simple word-count-example to illustrate the usage of the compatibility layer: -* the usage of spouts or bolt within a regular Flink streaming program (ie, embedded spouts or bolts) - 1. `SpoutSourceWordCount` uses a spout as data source within a Flink streaming program - 2. `BoltTokenizeerWordCount` uses a bolt to split sentences into words within a Flink streaming program - * `BoltTokenizeerWordCountWithNames` used Tuple input type and access attributes by field names (rather than index) - * `BoltTokenizeerWordCountPOJO` used POJO input type and access attributes by field names (rather then index) - -* how to submit a whole Storm topology to Flink - 3. `WordCountTopology` plugs a Storm topology together - * `StormWordCountLocal` submits the topology to a local Flink cluster (similiar to a `LocalCluster` in Storm) - (`StormWordCountNamedLocal` access attributes by field names rather than index) - * `StormWordCountRemoteByClient` submits the topology to a remote Flink cluster (simliar to the usage of `NimbusClient` in Storm) - * `StormWordCountRemoteBySubmitter` submits the topology to a remote Flink cluster (simliar to the usage of `StormSubmitter` in Storm) - -Additionally, this module package the three examples word-count programs as jar files to be submitted to a Flink cluster via `bin/flink run example.jar`. -(Valid jars are `WordCount-SpoutSource.jar`, `WordCount-BoltTokenizer.jar`, and `WordCount-StormTopology.jar`) - -The package `org.apache.flink.stormcompatiblitly.wordcount.stormoperators` contain original Storm spouts and bolts that can be used unmodified within Storm or Flink. diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java deleted file mode 100644 index 0f64301087df6..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.stormcompatibility.excamation; - -import backtype.storm.Config; -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.NotAliveException; -import backtype.storm.utils.Utils; -import org.apache.flink.stormcompatibility.api.FlinkClient; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; - -/** - * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text - * files in a streaming fashion. The program is constructed as a regular {@link StormTopology} and - * submitted to Flink for execution in the same way as to a Storm cluster similar to - * {@link NimbusClient}. The Flink cluster can be local or remote. - *

- * This example shows how to submit the program via Java, thus it cannot be used to submit a - * {@link StormTopology} via Flink command line clients (ie, bin/flink). - *

- *

- * The input is a plain text file with lines separated by newline characters. - *

- *

- * Usage: StormExclamationRemoteByClient <text path> <result path>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}. - *

- *

- * This example shows how to: - *

    - *
  • submit a regular Storm program to a local or remote Flink cluster.
  • - *
- */ -public class StormExclamationRemoteByClient { - - public final static String topologyId = "Streaming Exclamation"; - private final static String uploadedJarLocation = "target/flink-storm-examples-0.9-SNAPSHOT-ExclamationStorm.jar"; - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(final String[] args) throws AlreadyAliveException, InvalidTopologyException, - NotAliveException { - - if (!ExclamationTopology.parseParameters(args)) { - return; - } - - // build Topology the Storm way - final FlinkTopologyBuilder builder = ExclamationTopology.buildTopology(); - - // execute program on Flink cluster - final Config conf = new Config(); - // can be changed to remote address - conf.put(Config.NIMBUS_HOST, "localhost"); - // use default flink jobmanger.rpc.port - conf.put(Config.NIMBUS_THRIFT_PORT, 6123); - - final FlinkClient cluster = FlinkClient.getConfiguredClient(conf); - cluster.submitTopology(topologyId, uploadedJarLocation, builder.createTopology()); - - Utils.sleep(5 * 1000); - - cluster.killTopology(topologyId); - } -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java deleted file mode 100644 index d58052003597e..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.stormcompatibility.excamation; - -import backtype.storm.Config; -import org.apache.flink.stormcompatibility.api.FlinkClient; -import org.apache.flink.stormcompatibility.api.FlinkSubmitter; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; - -/** - * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text - * files in a streaming fashion. The program is constructed as a regular {@link StormTopology} and - * submitted to Flink for execution in the same way as to a Storm cluster similar to - * {@link StormSubmitter}. The Flink cluster can be local or remote. - *

- * This example shows how to submit the program via Java as well as Flink's command line client (ie, bin/flink). - *

- *

- * The input is a plain text file with lines separated by newline characters. - *

- *

- * Usage: StormExclamationRemoteByClient <text path> <result path>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}. - *

- *

- * This example shows how to: - *

    - *
  • submit a regular Storm program to a local or remote Flink cluster.
  • - *
- */ -public class StormExclamationRemoteBySubmitter { - - public final static String topologyId = "Streaming Exclamation"; - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(final String[] args) throws Exception { - - if (!ExclamationTopology.parseParameters(args)) { - return; - } - - // build Topology the Storm way - final FlinkTopologyBuilder builder = ExclamationTopology.buildTopology(); - - // execute program on Flink cluster - final Config conf = new Config(); - // We can set Jobmanager host/port values manually or leave them blank - // if not set and - // - executed within Java, default values "localhost" and "6123" are set by FlinkSubmitter - // - executed via bin/flink values from flink-conf.yaml are set by FlinkSubmitter. - // conf.put(Config.NIMBUS_HOST, "localhost"); - // conf.put(Config.NIMBUS_THRIFT_PORT, new Integer(6123)); - - // The user jar file must be specified via JVM argument if executed via Java. - // => -Dstorm.jar=target/flink-storm-examples-0.9-SNAPSHOT-WordCountStorm.jar - // If bin/flink is used, the jar file is detected automatically. - FlinkSubmitter.submitTopology(topologyId, conf, builder.createTopology()); - - Thread.sleep(5 * 1000); - - FlinkClient.getConfiguredClient(conf).killTopology(topologyId); - } -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/SingleJoinTopology.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/SingleJoinTopology.java deleted file mode 100644 index 79c71257c96fa..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/SingleJoinTopology.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * 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.stormcompatibility.singlejoin; - -import backtype.storm.tuple.Fields; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; -import org.apache.flink.stormcompatibility.singlejoin.stormoperators.AgeSpout; -import org.apache.flink.stormcompatibility.singlejoin.stormoperators.GenderSpout; -import org.apache.flink.stormcompatibility.singlejoin.stormoperators.SingleJoinBolt; -import org.apache.flink.stormcompatibility.util.OutputFormatter; -import org.apache.flink.stormcompatibility.util.StormBoltFileSink; -import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; -import org.apache.flink.stormcompatibility.util.TupleOutputFormatter; - -public class SingleJoinTopology { - - public final static String spoutId1 = "gender"; - public final static String spoutId2 = "age"; - public final static String boltId = "singleJoin"; - public final static String sinkId = "sink"; - private final static OutputFormatter formatter = new TupleOutputFormatter(); - - public static FlinkTopologyBuilder buildTopology() { - - final FlinkTopologyBuilder builder = new FlinkTopologyBuilder(); - - // get input data - builder.setSpout(spoutId1, new GenderSpout(new Fields("id", "gender"))); - builder.setSpout(spoutId2, new AgeSpout(new Fields("id", "age"))); - - builder.setBolt(boltId, new SingleJoinBolt(new Fields("gender", "age"))) - .fieldsGrouping(spoutId1, new Fields("id")) - .fieldsGrouping(spoutId2, new Fields("id")); - //.shuffleGrouping(spoutId1) - //.shuffleGrouping(spoutId2); - - // emit result - if (fileInputOutput) { - // read the text file from given input path - final String[] tokens = outputPath.split(":"); - final String outputFile = tokens[tokens.length - 1]; - builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)).shuffleGrouping(boltId); - } else { - builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4).shuffleGrouping(boltId); - } - - return builder; - } - - // ************************************************************************* - // UTIL METHODS - // ************************************************************************* - - private static boolean fileInputOutput = false; - private static String outputPath; - - static boolean parseParameters(final String[] args) { - - if (args.length > 0) { - // parse input arguments - fileInputOutput = true; - if (args.length == 1) { - outputPath = args[0]; - } else { - System.err.println("Usage: StormSingleJoin* "); - return false; - } - } else { - System.out.println("Executing StormSingleJoin* example with built-in default data"); - System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: StormSingleJoin* "); - } - return true; - } -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/StormSingleJoinLocal.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/StormSingleJoinLocal.java deleted file mode 100644 index d70914ab819a9..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/StormSingleJoinLocal.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.stormcompatibility.singlejoin; - -import backtype.storm.utils.Utils; -import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; - -public class StormSingleJoinLocal { - public final static String topologyId = "Streaming SingleJoin"; - - // ************************************************************************* - // PROGRAM - // ************************************************************************* - - public static void main(final String[] args) throws Exception { - - if (!SingleJoinTopology.parseParameters(args)) { - return; - } - - // build Topology the Storm way - final FlinkTopologyBuilder builder = SingleJoinTopology.buildTopology(); - - // execute program locally - final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster(); - cluster.submitTopology(topologyId, null, builder.createTopology()); - - Utils.sleep(5 * 1000); - - // TODO kill does no do anything so far - cluster.killTopology(topologyId); - cluster.shutdown(); - } -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/AgeSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/AgeSpout.java deleted file mode 100644 index 49761c3e4f258..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/AgeSpout.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.stormcompatibility.singlejoin.stormoperators; - -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import org.apache.flink.stormcompatibility.util.AbstractStormSpout; - -public class AgeSpout extends AbstractStormSpout { - private static final long serialVersionUID = -4008858647468647019L; - - private int counter = 0; - private String gender; - private Fields outFields; - - public AgeSpout(Fields outFields) { - this.outFields = outFields; - } - - @Override - public void nextTuple() { - if (this.counter < 10) { - if (counter % 2 == 0) { - gender = "male"; - } else { - gender = "female"; - } - this.collector.emit(new Values(counter, gender)); - counter++; - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(outFields); - } - -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/GenderSpout.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/GenderSpout.java deleted file mode 100644 index 238b6db68b189..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/GenderSpout.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.stormcompatibility.singlejoin.stormoperators; - -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import org.apache.flink.stormcompatibility.util.AbstractStormSpout; - -public class GenderSpout extends AbstractStormSpout { - private static final long serialVersionUID = -5079110197950743927L; - - private int counter = 9; - private Fields outFields; - - public GenderSpout(Fields outFields) { - this.outFields = outFields; - } - - @Override - public void nextTuple() { - if (counter >= 0) { - this.collector.emit(new Values(counter, counter + 20)); - counter--; - } - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(outFields); - } -} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/SingleJoinBolt.java b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/SingleJoinBolt.java deleted file mode 100644 index cd53140470f29..0000000000000 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/SingleJoinBolt.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.stormcompatibility.singlejoin.stormoperators; - -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.utils.TimeCacheMap; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -@SuppressWarnings("deprecation") -public class SingleJoinBolt implements IRichBolt { - OutputCollector collector; - Fields idFields; - Fields outFields; - int numSources = 2; - TimeCacheMap, Map> pending; - Map fieldLocations; - - public SingleJoinBolt(Fields outFields) { - this.outFields = outFields; - } - - @SuppressWarnings({"rawtypes", "null"}) - @Override - public void prepare(Map conf, TopologyContext context, OutputCollector collector) { - fieldLocations = new HashMap(); - this.collector = collector; - int timeout = 100; - pending = new TimeCacheMap, Map>(timeout, new ExpireCallback()); - // numSources = context.getThisSources().size(); - Set idFields = null; - for (GlobalStreamId source : context.getThisSources().keySet()) { - Fields fields = context.getComponentOutputFields(source.get_componentId(), source.get_streamId()); - Set setFields = new HashSet(fields.toList()); - if (idFields == null) { - idFields = setFields; - } else { - idFields.retainAll(setFields); - } - - for (String outfield : outFields) { - for (String sourcefield : fields) { - if (outfield.equals(sourcefield)) { - fieldLocations.put(outfield, source); - } - } - } - } - this.idFields = new Fields(new ArrayList(idFields)); - - if (fieldLocations.size() != outFields.size()) { - throw new RuntimeException("Cannot find all outfields among sources"); - } - } - - @Override - public void execute(Tuple tuple) { - List id = tuple.select(idFields); - GlobalStreamId streamId = new GlobalStreamId(tuple.getSourceComponent(), tuple.getSourceStreamId()); - if (!pending.containsKey(id)) { - pending.put(id, new HashMap()); - } - Map parts = pending.get(id); - if (parts.containsKey(streamId)) { - throw new RuntimeException("Received same side of single join twice"); - } - parts.put(streamId, tuple); - if (parts.size() == numSources) { - pending.remove(id); - List joinResult = new ArrayList(); - for (String outField : outFields) { - GlobalStreamId loc = fieldLocations.get(outField); - joinResult.add(parts.get(loc).getValueByField(outField)); - } - collector.emit(new ArrayList(parts.values()), joinResult); - - for (Tuple part : parts.values()) { - collector.ack(part); - } - } - } - - @Override - public void cleanup() { - /* nothing to do */ - } - - @Override - public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(outFields); - } - - @Override - public Map getComponentConfiguration() { - return null; - } - - private class ExpireCallback implements TimeCacheMap.ExpiredCallback, Map> { - @Override - public void expire(List id, Map tuples) { - for (Tuple tuple : tuples.values()) { - collector.fail(tuple); - } - } - } -} diff --git a/flink-contrib/flink-storm-compatibility/pom.xml b/flink-contrib/flink-storm-compatibility/pom.xml deleted file mode 100644 index 803336ffc2181..0000000000000 --- a/flink-contrib/flink-storm-compatibility/pom.xml +++ /dev/null @@ -1,40 +0,0 @@ - - - - - 4.0.0 - - - org.apache.flink - flink-contrib-parent - 0.10-SNAPSHOT - .. - - - flink-storm-compatibility-parent - flink-storm-compatibility - pom - - - flink-storm-compatibility-core - flink-storm-compatibility-examples - - diff --git a/flink-contrib/flink-storm-examples/README.md b/flink-contrib/flink-storm-examples/README.md new file mode 100644 index 0000000000000..c3247f650b6e1 --- /dev/null +++ b/flink-contrib/flink-storm-examples/README.md @@ -0,0 +1,20 @@ +# flink-storm-examples + +This module contains multiple versions of a simple Word-Count example to illustrate the usage of the compatibility layer: +* the usage of spouts and bolts within a regular Flink streaming program (ie, embedded mode) + 1. `SpoutSourceWordCount` uses a spout as data source within a Flink streaming program + 2. `BoltTokenizeerWordCount` uses a bolt to split sentences into words within a Flink streaming program + * `BoltTokenizeerWordCountWithNames` used `Tuple` input type and accesses attributes by field names (rather than index) + * `BoltTokenizeerWordCountPOJO` used POJO input type and accesses attributes by field names (rather than index) + +* how to submit a whole Storm topology to Flink + 3. `WordCountTopology` plugs a Storm topology together + * `StormWordCountLocal` submits the topology to a local Flink cluster (similiar to a `LocalCluster` in Storm) + (`WordCountLocalByName` accesses attributes by field names rather than index) + * `WordCountRemoteByClient` submits the topology to a remote Flink cluster (simliar to the usage of `NimbusClient` in Storm) + * `WordCountRemoteBySubmitter` submits the topology to a remote Flink cluster (simliar to the usage of `StormSubmitter` in Storm) + +Additionally, this module package the three example Word-Count programs as jar files to be submitted to a Flink cluster via `bin/flink run example.jar`. +(Valid jars are `WordCount-SpoutSource.jar`, `WordCount-BoltTokenizer.jar`, and `WordCount-StormTopology.jar`) + +The package `org.apache.flink.storm.wordcount.operators` contains original spouts and bolts that can be used unmodified within Storm or Flink. diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml b/flink-contrib/flink-storm-examples/pom.xml similarity index 80% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml rename to flink-contrib/flink-storm-examples/pom.xml index 2f3c02dfd5919..de6c0cbf011a3 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml +++ b/flink-contrib/flink-storm-examples/pom.xml @@ -24,20 +24,20 @@ under the License. org.apache.flink - flink-storm-compatibility-parent + flink-contrib-parent 0.10-SNAPSHOT .. - flink-storm-compatibility-examples - flink-storm-compatibility-examples + flink-storm-examples + flink-storm-examples jar org.apache.flink - flink-storm-compatibility-core + flink-storm ${project.version} @@ -90,7 +90,7 @@ under the License. org.apache.flink - flink-storm-compatibility-core + flink-storm ${project.version} jar false @@ -152,7 +152,7 @@ under the License. - org.apache.flink.stormcompatibility.wordcount.SpoutSourceWordCount + org.apache.flink.storm.wordcount.SpoutSourceWordCount @@ -168,16 +168,17 @@ under the License. org/json/simple/**/*.class - org/apache/flink/stormcompatibility/api/*.class - org/apache/flink/stormcompatibility/wrappers/*.class + org/apache/flink/storm/api/*.class + org/apache/flink/storm/util/*.class + org/apache/flink/storm/wrappers/*.class - org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.class - org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount$*.class - org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.class - org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.class - org/apache/flink/stormcompatibility/util/AbstractStormSpout.class - org/apache/flink/stormcompatibility/util/StormFileSpout.class - org/apache/flink/stormcompatibility/util/StormInMemorySpout.class + org/apache/flink/storm/wordcount/SpoutSourceWordCount.class + org/apache/flink/storm/wordcount/SpoutSourceWordCount$*.class + org/apache/flink/storm/wordcount/operators/WordCountFileSpout.class + org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.class + org/apache/flink/storm/util/AbstractLineSpout.class + org/apache/flink/storm/util/FileSpout.class + org/apache/flink/storm/util/InMemorySpout.class org/apache/flink/examples/java/wordcount/util/WordCountData.class @@ -197,7 +198,7 @@ under the License. - org.apache.flink.stormcompatibility.wordcount.BoltTokenizerWordCount + org.apache.flink.storm.wordcount.BoltTokenizerWordCount @@ -213,11 +214,12 @@ under the License. org/json/simple/**/*.class - org/apache/flink/stormcompatibility/api/*.class - org/apache/flink/stormcompatibility/wrappers/*.class + org/apache/flink/storm/api/*.class + org/apache/flink/storm/util/*.class + org/apache/flink/storm/wrappers/*.class - org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.class - org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.class + org/apache/flink/storm/wordcount/BoltTokenizerWordCount.class + org/apache/flink/storm/wordcount/operators/BoltTokenizer.class org/apache/flink/examples/java/wordcount/util/WordCountData.class @@ -256,7 +258,6 @@ under the License. maven-shade-plugin org.apache.maven.plugins - 2.4.1 WordCount-StormTopology @@ -273,8 +274,8 @@ under the License. org.yaml:snakeyaml com.googlecode.json-simple:json-simple - org.apache.flink:flink-storm-compatibility-core - org.apache.flink:flink-storm-compatibility-examples + org.apache.flink:flink-storm + org.apache.flink:flink-storm-examples @@ -299,26 +300,27 @@ under the License. - org.apache.flink:flink-storm-compatibility-examples + org.apache.flink:flink-storm-examples - org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.class - org/apache/flink/stormcompatibility/wordcount/WordCountTopology.class - org/apache/flink/stormcompatibility/wordcount/stormoperators/*.class - org/apache/flink/stormcompatibility/util/*.class + org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.class + org/apache/flink/storm/wordcount/WordCountTopology.class + org/apache/flink/storm/wordcount/operators/*.class + org/apache/flink/storm/util/*.class org/apache/flink/examples/java/wordcount/util/WordCountData.class - org.apache.flink:flink-storm-compatibility-core + org.apache.flink:flink-storm - org/apache/flink/stormcompatibility/api/*.class - org/apache/flink/stormcompatibility/wrappers/*.class + org/apache/flink/storm/api/*.class + org/apache/flink/storm/util/*.class + org/apache/flink/storm/wrappers/*.class - org.apache.flink.stormcompatibility.wordcount.StormWordCountRemoteBySubmitter + org.apache.flink.storm.wordcount.WordCountRemoteBySubmitter diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java similarity index 77% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java index c87b3a566cc97..985cd68fce8fb 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.excamation; +package org.apache.flink.storm.excamation; import backtype.storm.Config; import backtype.storm.utils.Utils; -import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; -import org.apache.flink.stormcompatibility.excamation.stormoperators.ExclamationBolt; +import org.apache.flink.storm.api.FlinkLocalCluster; +import org.apache.flink.storm.api.FlinkTopologyBuilder; +import org.apache.flink.storm.excamation.operators.ExclamationBolt; /** - * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text - * files in a streaming fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology} - * and submitted to Flink for execution in the same way as to a Storm {@link backtype.storm.LocalCluster}. + * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text files in a streaming + * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology} and submitted to + * Flink for execution in the same way as to a Storm {@link backtype.storm.LocalCluster}. *

* This example shows how to run program directly within Java, thus it cannot be used to submit a * {@link backtype.storm.generated.StormTopology} via Flink command line clients (ie, bin/flink). @@ -36,7 +36,7 @@ * The input is a plain text file with lines separated by newline characters. *

*

- * Usage: StormExclamationLocal <text path> <result path>
+ * Usage: ExclamationLocal <text path> <result path>
* If no parameters are provided, the program is run with default data from * {@link org.apache.flink.examples.java.wordcount.util.WordCountData}. *

@@ -46,7 +46,7 @@ *

  • run a regular Storm program locally on Flink
  • * */ -public class StormExclamationLocal { +public class ExclamationLocal { public final static String topologyId = "Streaming Exclamation"; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java similarity index 65% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java index d8d620b856a03..70d25a2b05048 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java @@ -15,36 +15,35 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.excamation; +package org.apache.flink.storm.excamation; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; -import org.apache.flink.stormcompatibility.excamation.stormoperators.ExclamationBolt; -import org.apache.flink.stormcompatibility.util.FiniteStormFileSpout; -import org.apache.flink.stormcompatibility.util.FiniteStormInMemorySpout; -import org.apache.flink.stormcompatibility.util.OutputFormatter; -import org.apache.flink.stormcompatibility.util.SimpleOutputFormatter; -import org.apache.flink.stormcompatibility.util.StormBoltFileSink; -import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; +import org.apache.flink.storm.api.FlinkTopologyBuilder; +import org.apache.flink.storm.excamation.operators.ExclamationBolt; +import org.apache.flink.storm.util.FiniteFileSpout; +import org.apache.flink.storm.util.FiniteInMemorySpout; +import org.apache.flink.storm.util.OutputFormatter; +import org.apache.flink.storm.util.SimpleOutputFormatter; +import org.apache.flink.storm.util.BoltFileSink; +import org.apache.flink.storm.util.BoltPrintSink; /** - * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text - * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. + * Implements the "Exclamation" program that attaches two exclamation marks to every line of a text files in a streaming + * fashion. The program is constructed as a regular {@link StormTopology}. *

    *

    * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: StormExclamation[Local|RemoteByClient|RemoteBySubmitter] <text path> + * Usage: Exclamation[Local|RemoteByClient|RemoteBySubmitter] <text path> * <result path>
    - * If no parameters are provided, the program is run with default data from - * {@link WordCountData}. + * If no parameters are provided, the program is run with default data from {@link WordCountData}. *

    *

    * This example shows how to: *

      *
    • construct a regular Storm topology as Flink program
    • - *
    • make use of the FiniteStormSpout interface
    • + *
    • make use of the FiniteSpout interface
    • *
    */ public class ExclamationTopology { @@ -63,9 +62,9 @@ public static FlinkTopologyBuilder buildTopology() { // read the text file from given input path final String[] tokens = textPath.split(":"); final String inputFile = tokens[tokens.length - 1]; - builder.setSpout(spoutId, new FiniteStormFileSpout(inputFile)); + builder.setSpout(spoutId, new FiniteFileSpout(inputFile)); } else { - builder.setSpout(spoutId, new FiniteStormInMemorySpout(WordCountData.WORDS)); + builder.setSpout(spoutId, new FiniteInMemorySpout(WordCountData.WORDS)); } builder.setBolt(firstBoltId, new ExclamationBolt(), 3).shuffleGrouping(spoutId); @@ -76,11 +75,11 @@ public static FlinkTopologyBuilder buildTopology() { // read the text file from given input path final String[] tokens = outputPath.split(":"); final String outputFile = tokens[tokens.length - 1]; - builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)) - .shuffleGrouping(secondBoltId); + builder.setBolt(sinkId, new BoltFileSink(outputFile, formatter)) + .shuffleGrouping(secondBoltId); } else { - builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4) - .shuffleGrouping(secondBoltId); + builder.setBolt(sinkId, new BoltPrintSink(formatter), 4) + .shuffleGrouping(secondBoltId); } return builder; @@ -109,17 +108,13 @@ static boolean parseParameters(final String[] args) { outputPath = args[1]; exclamationNum = Integer.parseInt(args[2]); } else { - System.err.println( - "Usage: StormExclamation[Local|RemoteByClient|RemoteBySubmitter] "); + System.err.println("Usage: StormExclamation* "); return false; } } else { System.out.println("Executing StormExclamation example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println( - " Usage: StormExclamation[Local|RemoteByClient|RemoteBySubmitter] " + - " "); + System.out.println(" Usage: StormExclamation "); } return true; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java similarity index 74% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java index c8af3a63b869b..01ab907d33947 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java @@ -16,38 +16,39 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.excamation; +package org.apache.flink.storm.excamation; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.excamation.stormoperators.ExclamationBolt; -import org.apache.flink.stormcompatibility.util.StormConfig; -import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; +import org.apache.flink.storm.excamation.operators.ExclamationBolt; +import org.apache.flink.storm.util.StormConfig; +import org.apache.flink.storm.wrappers.BoltWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import backtype.storm.utils.Utils; /** - * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text - * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. + * Implements the "Exclamation" program that attaches 3+x exclamation marks to every line of a text files in a streaming + * fashion. The program is constructed as a regular {@link StormTopology}. *

    *

    * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: StormExclamationWithStormBolt <text path> <result path> <number of exclamation marks>
    - * If no parameters are provided, the program is run with default data from - * {@link WordCountData}. + * Usage: + * ExclamationWithmBolt <text path> <result path> <number of exclamation marks>
    + * If no parameters are provided, the program is run with default data from {@link WordCountData} with x=2. *

    *

    * This example shows how to: *

      - *
    • use a Storm bolt within a Flink Streaming program
    • + *
    • use a Bolt within a Flink Streaming program
    • + *
    • how to configure a Bolt using StormConfig
    • *
    */ -public class ExclamationWithStormBolt { +public class ExclamationWithBolt { // ************************************************************************* // PROGRAM @@ -73,9 +74,9 @@ public static void main(final String[] args) throws Exception { final DataStream exclaimed = text .transform("StormBoltTokenizer", TypeExtractor.getForObject(""), - new StormBoltWrapper(new ExclamationBolt(), + new BoltWrapper(new ExclamationBolt(), new String[] { Utils.DEFAULT_STREAM_ID })) - .map(new ExclamationMap()); + .map(new ExclamationMap()); // emit result if (fileOutput) { @@ -85,7 +86,7 @@ public static void main(final String[] args) throws Exception { } // execute program - env.execute("Streaming WordCount with Storm bolt tokenizer"); + env.execute("Streaming WordCount with bolt tokenizer"); } // ************************************************************************* @@ -108,7 +109,7 @@ public String map(String value) throws Exception { private static boolean fileOutput = false; private static String textPath; private static String outputPath; - private static int exclamationNum = 3; + private static int exclamationNum = 2; private static boolean parseParameters(final String[] args) { @@ -120,13 +121,13 @@ private static boolean parseParameters(final String[] args) { outputPath = args[1]; exclamationNum = Integer.parseInt(args[2]); } else { - System.err.println("Usage: ExclamationWithStormBolt "); + System.err.println("Usage: ExclamationWithBolt "); return false; } } else { - System.out.println("Executing ExclamationWithStormBolt example with built-in default data"); + System.out.println("Executing ExclamationWithBolt example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: ExclamationWithStormBolt "); + System.out.println(" Usage: ExclamationWithBolt "); } return true; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java similarity index 76% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java index 99c816dffb9a5..22938e501a837 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java @@ -16,40 +16,41 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.excamation; +package org.apache.flink.storm.excamation; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.util.FiniteStormFileSpout; -import org.apache.flink.stormcompatibility.util.FiniteStormInMemorySpout; -import org.apache.flink.stormcompatibility.util.StormConfig; -import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; +import org.apache.flink.storm.util.FiniteFileSpout; +import org.apache.flink.storm.util.FiniteInMemorySpout; +import org.apache.flink.storm.util.StormConfig; +import org.apache.flink.storm.wrappers.SpoutWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import backtype.storm.utils.Utils; /** - * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text - * files in a streaming fashion. The program is constructed as a regular {@link StormTopology}. + * Implements the "Exclamation" program that attaches six exclamation marks to every line of a text files in a streaming + * fashion. The program is constructed as a regular {@link StormTopology}. *

    *

    * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: StormExclamationWithStormSpout <text path> <result path>
    - * If no parameters are provided, the program is run with default data from - * {@link WordCountData}. + * Usage: ExclamationWithSpout <text path> <result path>
    + * If no parameters are provided, the program is run with default data from {@link WordCountData}. *

    *

    * This example shows how to: *

      *
    • use a Storm spout within a Flink Streaming program
    • - *
    • make use of the FiniteStormSpout interface
    • + *
    • make use of the FiniteSpout interface
    • + *
    • make use of the FiniteSpout interface
    • + *
    • how to configure a Spout using StormConfig
    • *
    */ -public class ExclamationWithStormSpout { +public class ExclamationWithSpout { // ************************************************************************* // PROGRAM @@ -112,14 +113,13 @@ private static boolean parseParameters(final String[] args) { textPath = args[0]; outputPath = args[1]; } else { - System.err.println("Usage: ExclamationWithStormSpout "); + System.err.println("Usage: ExclamationWithSpout "); return false; } } else { - System.out.println("Executing ExclamationWithStormSpout example with built-in default " + - "data"); + System.out.println("Executing ExclamationWithSpout example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: ExclamationWithStormSpout "); + System.out.println(" Usage: ExclamationWithSpout "); } return true; } @@ -131,17 +131,17 @@ private static DataStream getTextDataStream(final StreamExecutionEnviron // set Storm configuration StormConfig config = new StormConfig(); - config.put(FiniteStormFileSpout.INPUT_FILE_PATH, inputFile); + config.put(FiniteFileSpout.INPUT_FILE_PATH, inputFile); env.getConfig().setGlobalJobParameters(config); return env.addSource( - new FiniteStormSpoutWrapper(new FiniteStormFileSpout(), + new SpoutWrapper(new FiniteFileSpout(), new String[] { Utils.DEFAULT_STREAM_ID }), TypeExtractor.getForClass(String.class)).setParallelism(1); } return env.addSource( - new FiniteStormSpoutWrapper(new FiniteStormInMemorySpout( + new SpoutWrapper(new FiniteInMemorySpout( WordCountData.WORDS), new String[] { Utils.DEFAULT_STREAM_ID }), TypeExtractor.getForClass(String.class)).setParallelism(1); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/stormoperators/ExclamationBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators/ExclamationBolt.java similarity index 96% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/stormoperators/ExclamationBolt.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators/ExclamationBolt.java index 2709effdf57f9..cfc49a1e1bce5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/stormoperators/ExclamationBolt.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators/ExclamationBolt.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.excamation.stormoperators; +package org.apache.flink.storm.excamation.operators; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java similarity index 80% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java index 18251d46644ac..560fe5169f345 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java @@ -15,30 +15,30 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.stormcompatibility.split.stormoperators.RandomSpout; -import org.apache.flink.stormcompatibility.split.stormoperators.VerifyAndEnrichBolt; -import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector; -import org.apache.flink.stormcompatibility.util.SplitStreamMapper; -import org.apache.flink.stormcompatibility.util.SplitStreamType; -import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; -import org.apache.flink.stormcompatibility.wrappers.StormSpoutWrapper; +import org.apache.flink.storm.split.operators.RandomSpout; +import org.apache.flink.storm.split.operators.VerifyAndEnrichBolt; +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.SpoutWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SplitStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** - * Implements a simple example with two declared output streams for the embedded Spout. + * Implements a simple example with two declared output streams for the embedded spout. *

    * This example shows how to: *

      *
    • handle multiple output stream of a spout
    • *
    • accessing each stream by .split(...) and .select(...)
    • - *
    • strip wrapper data type SplitStreamType for furhter processing in Flink
    • + *
    • strip wrapper data type SplitStreamType for further processing in Flink
    • *
    *

    * This example would work the same way for multiple bolt output streams. @@ -57,11 +57,11 @@ public static void main(final String[] args) throws Exception { String[] rawOutputs = new String[] { RandomSpout.EVEN_STREAM, RandomSpout.ODD_STREAM }; final DataStream> numbers = env.addSource( - new StormSpoutWrapper>(new RandomSpout(true, 0), + new SpoutWrapper>(new RandomSpout(true, 0), rawOutputs), TypeExtractor.getForObject(new SplitStreamType())); SplitStream> splitStream = numbers - .split(new FlinkStormStreamSelector()); + .split(new StormStreamSelector()); DataStream> evenStream = splitStream.select(RandomSpout.EVEN_STREAM); DataStream> oddStream = splitStream.select(RandomSpout.ODD_STREAM); @@ -69,7 +69,7 @@ public static void main(final String[] args) throws Exception { evenStream.map(new SplitStreamMapper()).returns(Integer.class).map(new Enrich("even")).print(); oddStream.transform("oddBolt", TypeExtractor.getForObject(new Tuple2("", 0)), - new StormBoltWrapper, Tuple2>( + new BoltWrapper, Tuple2>( new VerifyAndEnrichBolt(false))) .print(); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java index 75d710e37561f..d3153957beba5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split.stormoperators; +package org.apache.flink.storm.split.operators; import java.util.Map; import java.util.Random; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java similarity index 96% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java index 585370522810f..99fec4d83bfea 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split.stormoperators; +package org.apache.flink.storm.split.operators; import java.util.Map; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormBoltSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java similarity index 93% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormBoltSink.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java index b121744b4919b..a6c61d43f3e82 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormBoltSink.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; @@ -29,14 +29,14 @@ * Implements a sink that write the received data so some external output. The result is formatted like * {@code (a1, a2, ..., an)} with {@code Object.toString()} for each attribute). */ -public abstract class AbstractStormBoltSink implements IRichBolt { +public abstract class AbstractBoltSink implements IRichBolt { private static final long serialVersionUID = -1626323806848080430L; private StringBuilder lineBuilder; private String prefix = ""; private final OutputFormatter formatter; - public AbstractStormBoltSink(final OutputFormatter formatter) { + public AbstractBoltSink(final OutputFormatter formatter) { this.formatter = formatter; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java similarity index 86% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormSpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java index 4739a2c16cb01..d19ffbf0606ac 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; @@ -26,10 +26,10 @@ import java.util.Map; /** - * Base class for Storm Spout that reads data line by line from an arbitrary source. The declared output schema has a - * single attribute called {@code line} and should be of type {@link String}. + * Base class for Spouts that read data line by line from an arbitrary source. The declared output schema has a single + * attribute called {@code line} and should be of type {@link String}. */ -public abstract class AbstractStormSpout implements IRichSpout { +public abstract class AbstractLineSpout implements IRichSpout { private static final long serialVersionUID = 8876828403487806771L; public final static String ATTRIBUTE_LINE = "line"; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltFileSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java similarity index 88% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltFileSink.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java index ee8dca44d1d1c..5cd3f68b86f1c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltFileSink.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.task.TopologyContext; @@ -28,17 +28,17 @@ * Implements a sink that write the received data to the given file (as a result of {@code Object.toString()} for each * attribute). */ -public final class StormBoltFileSink extends AbstractStormBoltSink { +public final class BoltFileSink extends AbstractBoltSink { private static final long serialVersionUID = 2014027288631273666L; private final String path; private BufferedWriter writer; - public StormBoltFileSink(final String path) { + public BoltFileSink(final String path) { this(path, new SimpleOutputFormatter()); } - public StormBoltFileSink(final String path, final OutputFormatter formatter) { + public BoltFileSink(final String path, final OutputFormatter formatter) { super(formatter); this.path = path; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltPrintSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java similarity index 87% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltPrintSink.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java index 3bf49d06b2d19..044246b114c90 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltPrintSink.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.task.TopologyContext; @@ -24,10 +24,10 @@ /** * Implements a sink that prints the received data to {@code stdout}. */ -public final class StormBoltPrintSink extends AbstractStormBoltSink { +public final class BoltPrintSink extends AbstractBoltSink { private static final long serialVersionUID = -6650011223001009519L; - public StormBoltPrintSink(OutputFormatter formatter) { + public BoltPrintSink(OutputFormatter formatter) { super(formatter); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormFileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java similarity index 89% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormFileSpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java index 0611e3767c132..1126a2a0c8d6b 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormFileSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; @@ -28,9 +28,9 @@ import java.util.Map; /** - * Implements a Storm Spout that reads data from a given local file. + * Implements a Spout that reads data from a given local file. */ -public class StormFileSpout extends AbstractStormSpout { +public class FileSpout extends AbstractLineSpout { private static final long serialVersionUID = -6996907090003590436L; public final static String INPUT_FILE_PATH = "input.path"; @@ -38,9 +38,9 @@ public class StormFileSpout extends AbstractStormSpout { protected String path = null; protected BufferedReader reader; - public StormFileSpout() {} + public FileSpout() {} - public StormFileSpout(final String path) { + public FileSpout(final String path) { this.path = path; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java similarity index 83% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java index d3776fb47311f..75450c4f5262b 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java @@ -14,30 +14,30 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; import backtype.storm.tuple.Values; -import org.apache.flink.stormcompatibility.util.FiniteStormSpout; - import java.io.IOException; import java.util.Map; +import org.apache.flink.storm.util.FiniteSpout; + /** - * Implements a Storm Spout that reads data from a given local file. The spout stops automatically + * Implements a Spout that reads data from a given local file. The spout stops automatically * when it reached the end of the file. */ -public class FiniteStormFileSpout extends StormFileSpout implements FiniteStormSpout { +public class FiniteFileSpout extends FileSpout implements FiniteSpout { private static final long serialVersionUID = -1472978008607215864L; private String line; private boolean newLineRead; - public FiniteStormFileSpout() {} + public FiniteFileSpout() {} - public FiniteStormFileSpout(String path) { + public FiniteFileSpout(String path) { super(path); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java similarity index 70% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java index 5efff66b90494..14908729508d2 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java @@ -16,19 +16,19 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; + +import org.apache.flink.storm.util.FiniteSpout; -import org.apache.flink.stormcompatibility.util.FiniteStormSpout; /** - * Implements a Storm Spout that reads String[] data stored in the memory. The spout stops - * automatically when it emitted all of the data. + * Implements a Spout that reads String[] data stored in memory. The Spout stops automatically when it emitted all of + * the data. */ -public class FiniteStormInMemorySpout extends StormInMemorySpout implements - FiniteStormSpout { +public class FiniteInMemorySpout extends InMemorySpout implements FiniteSpout { private static final long serialVersionUID = -4008858647468647019L; - public FiniteStormInMemorySpout(String[] source) { + public FiniteInMemorySpout(String[] source) { super(source); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormInMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java similarity index 84% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormInMemorySpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java index f6ae622aa29a9..5e4c7ba8f9155 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormInMemorySpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.tuple.Values; /** - * Implements a Storm Spout that reads data from an in. + * Implements a Spout that reads data stored in memory. */ -public class StormInMemorySpout extends AbstractStormSpout { +public class InMemorySpout extends AbstractLineSpout { private static final long serialVersionUID = -4008858647468647019L; protected T[] source; protected int counter = 0; - public StormInMemorySpout(T[] source) { + public InMemorySpout(T[] source) { this.source = source; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java similarity index 84% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java index ec9adfe841ec7..e696f9bc2dac5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.tuple.Tuple; @@ -25,10 +25,11 @@ public interface OutputFormatter extends Serializable { /** - * Converts a Storm {@link Tuple} to a string. This method is used for formatting the output - * tuples before writing them out to a file or to the consol. - * - * @param input The tuple to be formatted + * Converts a Storm {@link Tuple} to a string. This method is used for formatting the output tuples before writing + * them out to a file or to the console. + * + * @param input + * The tuple to be formatted * @return The string result of the formatting */ public String format(Tuple input); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java similarity index 84% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java index 0702e947ea083..cef008105b4f8 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.tuple.Tuple; @@ -24,12 +24,11 @@ public class SimpleOutputFormatter implements OutputFormatter { private static final long serialVersionUID = 6349573860144270338L; /** - * Converts a Storm {@link Tuple} with 1 field to a string by retrieving the value of that - * field. This method is used for formatting raw outputs wrapped in tuples, before writing them - * out to a file or to the consol. - * + * Converts a Storm {@link Tuple} with 1 field to a string by retrieving the value of that field. This method is + * used for formatting raw outputs wrapped in tuples, before writing them out to a file or to the console. + * * @param input - * The tuple to be formatted + * The tuple to be formatted * @return The string result of the formatting */ @Override diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/TupleOutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java similarity index 96% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/TupleOutputFormatter.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java index 6419ee35d20e2..5d7ba53eaa535 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/TupleOutputFormatter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.tuple.Tuple; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java similarity index 84% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java index 6f7b6fb6d8ce8..aa3a075316ec3 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java @@ -15,20 +15,21 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.topology.IRichBolt; + import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormBoltTokenizer; -import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; +import org.apache.flink.storm.wordcount.operators.BoltTokenizer; +import org.apache.flink.storm.wrappers.BoltWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming - * fashion. The tokenizer step is performed by a Storm {@link IRichBolt bolt}. + * fashion. The tokenizer step is performed by a {@link IRichBolt Bolt}. *

    *

    * The input is a plain text file with lines separated by newline characters. @@ -40,7 +41,7 @@ *

    * This example shows how to: *

      - *
    • use a Storm bolt within a Flink Streaming program.
    • + *
    • use a Bolt within a Flink Streaming program.
    • *
    */ public class BoltTokenizerWordCount { @@ -63,11 +64,10 @@ public static void main(final String[] args) throws Exception { final DataStream> counts = text // split up the lines in pairs (2-tuples) containing: (word,1) - // this is done by a Storm bolt that is wrapped accordingly - .transform("StormBoltTokenizer", + // this is done by a bolt that is wrapped accordingly + .transform("BoltTokenizer", TypeExtractor.getForObject(new Tuple2("", 0)), - new StormBoltWrapper>(new StormBoltTokenizer())) - // split up the lines in pairs (2-tuples) containing: (word,1) + new BoltWrapper>(new BoltTokenizer())) // group by the tuple field "0" and sum up tuple field "1" .keyBy(0).sum(1); @@ -79,7 +79,7 @@ public static void main(final String[] args) throws Exception { } // execute program - env.execute("Streaming WordCount with Storm bolt tokenizer"); + env.execute("Streaming WordCount with bolt tokenizer"); } // ************************************************************************* diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java similarity index 74% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java index 300f5bcc9b465..f72acb333863f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.topology.IRichBolt; @@ -25,17 +25,17 @@ import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.Path; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormBoltTokenizerByName; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.WordCountDataPojos; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.WordCountDataPojos.Sentence; -import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; +import org.apache.flink.storm.wordcount.operators.BoltTokenizerByName; +import org.apache.flink.storm.wordcount.operators.WordCountDataPojos; +import org.apache.flink.storm.wordcount.operators.WordCountDataPojos.Sentence; +import org.apache.flink.storm.wrappers.BoltWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming - * fashion. The tokenizer step is performed by a Storm {@link IRichBolt bolt}. In contrast to - * {@link BoltTokenizerWordCount} the tokenizer's input is a POJO type and the single field is accessed by name. + * fashion. The tokenizer step is performed by a {@link IRichBolt Bolt}. In contrast to {@link BoltTokenizerWordCount} + * the tokenizer's input is a POJO type and the single field is accessed by name. *

    *

    * The input is a plain text file with lines separated by newline characters. @@ -47,7 +47,7 @@ *

    * This example shows how to: *

      - *
    • how to access attributes by name for POJO type input streams + *
    • how to access attributes by name within a Bolt for POJO type input streams *
    */ public class BoltTokenizerWordCountPojo { @@ -70,12 +70,10 @@ public static void main(final String[] args) throws Exception { final DataStream> counts = text // split up the lines in pairs (2-tuples) containing: (word,1) - // this is done by a Storm bolt that is wrapped accordingly - .transform("StormBoltTokenizer", + // this is done by a bolt that is wrapped accordingly + .transform("BoltTokenizerPojo", TypeExtractor.getForObject(new Tuple2("", 0)), - new StormBoltWrapper>( - new StormBoltTokenizerByName())) - // split up the lines in pairs (2-tuples) containing: (word,1) + new BoltWrapper>(new BoltTokenizerByName())) // group by the tuple field "0" and sum up tuple field "1" .keyBy(0).sum(1); @@ -87,7 +85,7 @@ public static void main(final String[] args) throws Exception { } // execute program - env.execute("Streaming WordCount with Storm bolt tokenizer"); + env.execute("Streaming WordCount with POJO bolt tokenizer"); } // ************************************************************************* @@ -107,13 +105,14 @@ private static boolean parseParameters(final String[] args) { textPath = args[0]; outputPath = args[1]; } else { - System.err.println("Usage: BoltTokenizerWordCount "); + System.err.println("Usage: BoltTokenizerWordCountPojo "); return false; } } else { - System.out.println("Executing BoltTokenizerWordCount example with built-in default data"); + System.out + .println("Executing BoltTokenizerWordCountPojo example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: BoltTokenizerWordCount "); + System.out.println(" Usage: BoltTokenizerWordCountPojo "); } return true; } @@ -121,7 +120,7 @@ private static boolean parseParameters(final String[] args) { private static DataStream getTextDataStream(final StreamExecutionEnvironment env) { if (fileOutput) { // read the text file from given input path - PojoTypeInfo sourceType = (PojoTypeInfo)TypeExtractor + PojoTypeInfo sourceType = (PojoTypeInfo) TypeExtractor .getForObject(new Sentence("")); return env.createInput(new CsvInputFormat(new Path( textPath), CsvInputFormat.DEFAULT_LINE_DELIMITER, diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java similarity index 78% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java index ed01181a4fa2f..7617e95814275 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.topology.IRichBolt; import backtype.storm.tuple.Fields; @@ -28,17 +28,16 @@ import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.Path; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormBoltTokenizerByName; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.WordCountDataTuple; -import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; +import org.apache.flink.storm.wordcount.operators.BoltTokenizerByName; +import org.apache.flink.storm.wordcount.operators.WordCountDataTuple; +import org.apache.flink.storm.wrappers.BoltWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming - * fashion. The tokenizer step is performed by a Storm {@link IRichBolt bolt}. In contrast to - * {@link BoltTokenizerWordCount} the tokenizer's input is a {@link Tuple} type and the single field is accessed by - * name. + * fashion. The tokenizer step is performed by a {@link IRichBolt Bolt}. In contrast to {@link BoltTokenizerWordCount} + * the tokenizer's input is a {@link Tuple} type and the single field is accessed by name. *

    *

    * The input is a plain text file with lines separated by newline characters. @@ -50,7 +49,7 @@ *

    * This example shows how to: *

      - *
    • how to access attributes by name for {@link Tuple} type input streams + *
    • how to access attributes by name within a Bolt for {@link Tuple} type input streams *
    */ public class BoltTokenizerWordCountWithNames { @@ -74,11 +73,11 @@ public static void main(final String[] args) throws Exception { final DataStream> counts = text // split up the lines in pairs (2-tuples) containing: (word,1) // this is done by a Storm bolt that is wrapped accordingly - .transform("StormBoltTokenizer", + .transform( + "BoltTokenizerWithNames", TypeExtractor.getForObject(new Tuple2("", 0)), - new StormBoltWrapper, Tuple2>( - new StormBoltTokenizerByName(), new Fields("sentence"))) - // split up the lines in pairs (2-tuples) containing: (word,1) + new BoltWrapper, Tuple2>( + new BoltTokenizerByName(), new Fields("sentence"))) // group by the tuple field "0" and sum up tuple field "1" .keyBy(0).sum(1); @@ -90,7 +89,7 @@ public static void main(final String[] args) throws Exception { } // execute program - env.execute("Streaming WordCount with Storm bolt tokenizer"); + env.execute("Streaming WordCount with schema bolt tokenizer"); } // ************************************************************************* @@ -110,13 +109,13 @@ private static boolean parseParameters(final String[] args) { textPath = args[0]; outputPath = args[1]; } else { - System.err.println("Usage: BoltTokenizerWordCount "); + System.err.println("Usage: BoltTokenizerWordCountWithNames "); return false; } } else { - System.out.println("Executing BoltTokenizerWordCount example with built-in default data"); + System.out.println("Executing BoltTokenizerWordCountWithNames example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: BoltTokenizerWordCount "); + System.out.println(" Usage: BoltTokenizerWordCountWithNames "); } return true; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java similarity index 84% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java index 21d7811fa5436..bb451fee666b5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.topology.IRichSpout; import backtype.storm.utils.Utils; @@ -24,16 +24,16 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormWordCountFileSpout; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormWordCountInMemorySpout; -import org.apache.flink.stormcompatibility.wrappers.StormFiniteSpoutWrapper; +import org.apache.flink.storm.wordcount.operators.WordCountFileSpout; +import org.apache.flink.storm.wordcount.operators.WordCountInMemorySpout; +import org.apache.flink.storm.wrappers.SpoutWrapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming - * fashion. The used data source is a Storm {@link IRichSpout bolt}. + * fashion. The used data source is a {@link IRichSpout Spout}. *

    *

    * The input is a plain text file with lines separated by newline characters. @@ -45,7 +45,7 @@ *

    * This example shows how to: *

      - *
    • use a Storm spout within a Flink Streaming program.
    • + *
    • use a Spout within a Flink Streaming program.
    • *
    */ public class SpoutSourceWordCount { @@ -80,7 +80,7 @@ public static void main(final String[] args) throws Exception { } // execute program - env.execute("Streaming WordCount with Storm spout source"); + env.execute("Streaming WordCount with spout source"); } // ************************************************************************* @@ -142,15 +142,15 @@ private static DataStream getTextDataStream(final StreamExecutionEnviron final String[] tokens = textPath.split(":"); final String localFile = tokens[tokens.length - 1]; return env.addSource( - new StormFiniteSpoutWrapper(new StormWordCountFileSpout(localFile), - new String[] { Utils.DEFAULT_STREAM_ID }), - TypeExtractor.getForClass(String.class)).setParallelism(1); + new SpoutWrapper(new WordCountFileSpout(localFile), + new String[] { Utils.DEFAULT_STREAM_ID }, -1), + TypeExtractor.getForClass(String.class)).setParallelism(1); } return env.addSource( - new StormFiniteSpoutWrapper(new StormWordCountInMemorySpout(), - new String[] { Utils.DEFAULT_STREAM_ID }), - TypeExtractor.getForClass(String.class)).setParallelism(1); + new SpoutWrapper(new WordCountInMemorySpout(), + new String[] { Utils.DEFAULT_STREAM_ID }, -1), + TypeExtractor.getForClass(String.class)).setParallelism(1); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java similarity index 87% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java index 836c8e945c13c..18f49c13f4584 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java @@ -15,14 +15,15 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.LocalCluster; import backtype.storm.generated.StormTopology; import backtype.storm.utils.Utils; + import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.storm.api.FlinkLocalCluster; +import org.apache.flink.storm.api.FlinkTopologyBuilder; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming @@ -36,7 +37,7 @@ * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: WordCount <text path> <result path>
    + * Usage: WordCountLocal <text path> <result path>
    * If no parameters are provided, the program is run with default data from {@link WordCountData}. *

    *

    @@ -45,8 +46,8 @@ *

  • run a regular Storm program locally on Flink
  • * */ -public class StormWordCountLocal { - public final static String topologyId = "Streaming WordCount"; +public class WordCountLocal { + public final static String topologyId = "Storm WordCount"; // ************************************************************************* // PROGRAM diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountNamedLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java similarity index 85% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountNamedLocal.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java index f51afab849e62..71a5e8debe2b7 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountNamedLocal.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java @@ -15,19 +15,20 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.LocalCluster; import backtype.storm.generated.StormTopology; import backtype.storm.utils.Utils; + import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.storm.api.FlinkLocalCluster; +import org.apache.flink.storm.api.FlinkTopologyBuilder; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming * fashion. The program is constructed as a regular {@link StormTopology} and submitted to Flink for execution in the - * same way as to a Storm {@link LocalCluster}. In contrast to {@link StormWordCountLocal} all bolts access the field of + * same way as to a Storm {@link LocalCluster}. In contrast to {@link WordCountLocal} all bolts access the field of * input tuples by name instead of index. *

    * This example shows how to run program directly within Java, thus it cannot be used to submit a {@link StormTopology} @@ -37,7 +38,7 @@ * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: WordCount <text path> <result path>
    + * Usage: WordCountLocalByName <text path> <result path>
    * If no parameters are provided, the program is run with default data from {@link WordCountData}. *

    *

    @@ -46,8 +47,8 @@ *

  • run a regular Storm program locally on Flink * */ -public class StormWordCountNamedLocal { - public final static String topologyId = "Streaming WordCountName"; +public class WordCountLocalByName { + public final static String topologyId = "Storm WordCountName"; // ************************************************************************* // PROGRAM diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java similarity index 88% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java index 3c79edab8380f..2e4fb03bd3d4d 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.Config; import backtype.storm.generated.AlreadyAliveException; @@ -24,9 +24,10 @@ import backtype.storm.generated.StormTopology; import backtype.storm.utils.NimbusClient; import backtype.storm.utils.Utils; + import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.api.FlinkClient; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.storm.api.FlinkClient; +import org.apache.flink.storm.api.FlinkTopologyBuilder; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming @@ -40,7 +41,7 @@ * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: WordCount <text path> <result path>
    + * Usage: WordCountRemoteByClient <text path> <result path>
    * If no parameters are provided, the program is run with default data from {@link WordCountData}. *

    *

    @@ -49,8 +50,8 @@ *

  • submit a regular Storm program to a local or remote Flink cluster.
  • * */ -public class StormWordCountRemoteByClient { - public final static String topologyId = "Streaming WordCount"; +public class WordCountRemoteByClient { + public final static String topologyId = "Storm WordCount"; private final static String uploadedJarLocation = "target/WordCount-StormTopology.jar"; // ************************************************************************* @@ -58,7 +59,7 @@ public class StormWordCountRemoteByClient { // ************************************************************************* public static void main(final String[] args) throws AlreadyAliveException, InvalidTopologyException, - NotAliveException { + NotAliveException { if (!WordCountTopology.parseParameters(args)) { return; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java similarity index 87% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java index de84f55dd1876..173074c8707a0 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java @@ -15,15 +15,16 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.Config; import backtype.storm.StormSubmitter; import backtype.storm.generated.StormTopology; + import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.api.FlinkClient; -import org.apache.flink.stormcompatibility.api.FlinkSubmitter; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.storm.api.FlinkClient; +import org.apache.flink.storm.api.FlinkSubmitter; +import org.apache.flink.storm.api.FlinkTopologyBuilder; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming @@ -36,7 +37,7 @@ * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: WordCount <text path> <result path>
    + * Usage: WordCountRemoteBySubmitter <text path> <result path>
    * If no parameters are provided, the program is run with default data from {@link WordCountData}. *

    *

    @@ -45,8 +46,8 @@ *

  • submit a regular Storm program to a local or remote Flink cluster.
  • * */ -public class StormWordCountRemoteBySubmitter { - public final static String topologyId = "Streaming WordCount"; +public class WordCountRemoteBySubmitter { + public final static String topologyId = "Storm WordCount"; // ************************************************************************* // PROGRAM diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java similarity index 62% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java index 45be82122dc05..8ee374ddb6b10 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java @@ -15,23 +15,23 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; import backtype.storm.generated.StormTopology; import backtype.storm.tuple.Fields; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; -import org.apache.flink.stormcompatibility.util.OutputFormatter; -import org.apache.flink.stormcompatibility.util.StormBoltFileSink; -import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; -import org.apache.flink.stormcompatibility.util.TupleOutputFormatter; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormBoltCounter; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormBoltCounterByName; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormBoltTokenizer; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormBoltTokenizerByName; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormWordCountFileSpout; -import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormWordCountInMemorySpout; +import org.apache.flink.storm.api.FlinkTopologyBuilder; +import org.apache.flink.storm.util.OutputFormatter; +import org.apache.flink.storm.util.BoltFileSink; +import org.apache.flink.storm.util.BoltPrintSink; +import org.apache.flink.storm.util.TupleOutputFormatter; +import org.apache.flink.storm.wordcount.operators.BoltCounter; +import org.apache.flink.storm.wordcount.operators.BoltCounterByName; +import org.apache.flink.storm.wordcount.operators.BoltTokenizer; +import org.apache.flink.storm.wordcount.operators.BoltTokenizerByName; +import org.apache.flink.storm.wordcount.operators.WordCountFileSpout; +import org.apache.flink.storm.wordcount.operators.WordCountInMemorySpout; /** * Implements the "WordCount" program that computes a simple word occurrence histogram over text files in a streaming @@ -41,7 +41,8 @@ * The input is a plain text file with lines separated by newline characters. *

    *

    - * Usage: WordCount <text path> <result path>
    + * Usage: + * WordCount[Local|LocalByName|RemoteByClient|RemoteBySubmitter] <text path> <result path>
    * If no parameters are provided, the program is run with default data from {@link WordCountData}. *

    *

    @@ -70,24 +71,24 @@ public static FlinkTopologyBuilder buildTopology(boolean indexOrName) { // read the text file from given input path final String[] tokens = textPath.split(":"); final String inputFile = tokens[tokens.length - 1]; - builder.setSpout(spoutId, new StormWordCountFileSpout(inputFile)); + builder.setSpout(spoutId, new WordCountFileSpout(inputFile)); } else { - builder.setSpout(spoutId, new StormWordCountInMemorySpout()); + builder.setSpout(spoutId, new WordCountInMemorySpout()); } if (indexOrName) { // split up the lines in pairs (2-tuples) containing: (word,1) - builder.setBolt(tokenierzerId, new StormBoltTokenizer(), 4).shuffleGrouping(spoutId); + builder.setBolt(tokenierzerId, new BoltTokenizer(), 4).shuffleGrouping(spoutId); // group by the tuple field "0" and sum up tuple field "1" - builder.setBolt(counterId, new StormBoltCounter(), 4).fieldsGrouping(tokenierzerId, - new Fields(StormBoltTokenizer.ATTRIBUTE_WORD)); + builder.setBolt(counterId, new BoltCounter(), 4).fieldsGrouping(tokenierzerId, + new Fields(BoltTokenizer.ATTRIBUTE_WORD)); } else { // split up the lines in pairs (2-tuples) containing: (word,1) - builder.setBolt(tokenierzerId, new StormBoltTokenizerByName(), 4).shuffleGrouping( + builder.setBolt(tokenierzerId, new BoltTokenizerByName(), 4).shuffleGrouping( spoutId); // group by the tuple field "0" and sum up tuple field "1" - builder.setBolt(counterId, new StormBoltCounterByName(), 4).fieldsGrouping( - tokenierzerId, new Fields(StormBoltTokenizerByName.ATTRIBUTE_WORD)); + builder.setBolt(counterId, new BoltCounterByName(), 4).fieldsGrouping( + tokenierzerId, new Fields(BoltTokenizerByName.ATTRIBUTE_WORD)); } // emit result @@ -95,9 +96,9 @@ public static FlinkTopologyBuilder buildTopology(boolean indexOrName) { // read the text file from given input path final String[] tokens = outputPath.split(":"); final String outputFile = tokens[tokens.length - 1]; - builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)).shuffleGrouping(counterId); + builder.setBolt(sinkId, new BoltFileSink(outputFile, formatter)).shuffleGrouping(counterId); } else { - builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4).shuffleGrouping(counterId); + builder.setBolt(sinkId, new BoltPrintSink(formatter), 4).shuffleGrouping(counterId); } return builder; @@ -120,13 +121,13 @@ static boolean parseParameters(final String[] args) { textPath = args[0]; outputPath = args[1]; } else { - System.err.println("Usage: StormWordCount* "); + System.err.println("Usage: WordCount* "); return false; } } else { - System.out.println("Executing StormWordCount* example with built-in default data"); + System.out.println("Executing WordCount example with built-in default data"); System.out.println(" Provide parameters to read input data from a file"); - System.out.println(" Usage: StormWordCount* "); + System.out.println(" Usage: WordCount* "); } return true; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounterByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java similarity index 80% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounterByName.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java index bf940c3f77189..d21a58435eadf 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounterByName.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; @@ -29,11 +29,13 @@ import java.util.Map; /** - * Implements the word counter that the occurrence of each unique word. The bolt takes a pair (input tuple schema: - * {@code }) and sums the given word count for each unique word (output tuple schema: + * Implements the word counter that counts the occurrence of each unique word. The bolt takes a pair (input tuple + * schema: {@code }) and sums the given word count for each unique word (output tuple schema: * {@code } ). + *

    + * Same as {@link BoltCounterByName}, but accesses input attribute by index (instead of name). */ -public class StormBoltCounterByName implements IRichBolt { +public class BoltCounter implements IRichBolt { private static final long serialVersionUID = 399619605462625934L; public static final String ATTRIBUTE_WORD = "word"; @@ -50,14 +52,14 @@ public void prepare(final Map stormConf, final TopologyContext context, final Ou @Override public void execute(final Tuple input) { - final String word = input.getStringByField(StormBoltTokenizer.ATTRIBUTE_WORD); + final String word = input.getString(BoltTokenizer.ATTRIBUTE_WORD_INDEX); Count currentCount = this.counts.get(word); if (currentCount == null) { currentCount = new Count(); this.counts.put(word, currentCount); } - currentCount.count += input.getIntegerByField(StormBoltTokenizer.ATTRIBUTE_COUNT); + currentCount.count += input.getInteger(BoltTokenizer.ATTRIBUTE_COUNT_INDEX); this.collector.emit(new Values(word, currentCount.count)); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java similarity index 80% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounter.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java index 1544c191c5e73..d5c05d7937346 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; @@ -29,11 +29,13 @@ import java.util.Map; /** - * Implements the word counter that the occurrence of each unique word. The bolt takes a pair (input tuple schema: - * {@code }) and sums the given word count for each unique word (output tuple schema: + * Implements the word counter that counts the occurrence of each unique word. The bolt takes a pair (input tuple + * schema: {@code }) and sums the given word count for each unique word (output tuple schema: * {@code } ). + *

    + * Same as {@link BoltCounter}, but accesses input attribute by name (instead of index). */ -public class StormBoltCounter implements IRichBolt { +public class BoltCounterByName implements IRichBolt { private static final long serialVersionUID = 399619605462625934L; public static final String ATTRIBUTE_WORD = "word"; @@ -50,14 +52,14 @@ public void prepare(final Map stormConf, final TopologyContext context, final Ou @Override public void execute(final Tuple input) { - final String word = input.getString(StormBoltTokenizer.ATTRIBUTE_WORD_INDEX); + final String word = input.getStringByField(BoltTokenizer.ATTRIBUTE_WORD); Count currentCount = this.counts.get(word); if (currentCount == null) { currentCount = new Count(); this.counts.put(word, currentCount); } - currentCount.count += input.getInteger(StormBoltTokenizer.ATTRIBUTE_COUNT_INDEX); + currentCount.count += input.getIntegerByField(BoltTokenizer.ATTRIBUTE_COUNT); this.collector.emit(new Values(word, currentCount.count)); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java similarity index 89% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java index dfb3e37d8f4c8..74d6a991d9da0 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; @@ -28,13 +28,13 @@ import java.util.Map; /** - * Implements the string tokenizer that splits sentences into words as a Storm bolt. The bolt takes a line (input tuple + * Implements the string tokenizer that splits sentences into words as a bolt. The bolt takes a line (input tuple * schema: {@code }) and splits it into multiple pairs in the form of "(word,1)" (output tuple schema: * {@code }). *

    - * Same as {@link StormBoltTokenizerByName}, but accesses input attribute by index (instead of name). + * Same as {@link BoltTokenizerByName}, but accesses input attribute by index (instead of name). */ -public final class StormBoltTokenizer implements IRichBolt { +public final class BoltTokenizer implements IRichBolt { private static final long serialVersionUID = -8589620297208175149L; public static final String ATTRIBUTE_WORD = "word"; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizerByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java similarity index 89% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizerByName.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java index 8796b952446b4..3c56b3608a439 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizerByName.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; @@ -28,13 +28,13 @@ import java.util.Map; /** - * Implements the string tokenizer that splits sentences into words as a Storm bolt. The bolt takes a line (input tuple + * Implements the string tokenizer that splits sentences into words as a bolt. The bolt takes a line (input tuple * schema: {@code }) and splits it into multiple pairs in the form of "(word,1)" (output tuple schema: * {@code }). *

    - * Same as {@link StormBoltTokenizer}, but accesses input attribute by name (instead of index). + * Same as {@link BoltTokenizer}, but accesses input attribute by name (instead of index). */ -public final class StormBoltTokenizerByName implements IRichBolt { +public final class BoltTokenizerByName implements IRichBolt { private static final long serialVersionUID = -8589620297208175149L; public static final String ATTRIBUTE_WORD = "word"; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataPojos.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataPojos.java similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataPojos.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataPojos.java index f965a2809c09f..3a8fd3ad5716f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataPojos.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataPojos.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; import java.io.Serializable; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataTuple.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataTuple.java similarity index 94% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataTuple.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataTuple.java index 732f0aecaa344..16e2ba00f77fb 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataTuple.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataTuple.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.examples.java.wordcount.util.WordCountData; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java similarity index 77% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java index e99476085146a..76a198f80b474 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; -import org.apache.flink.stormcompatibility.util.StormFileSpout; +import org.apache.flink.storm.util.FileSpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; /** - * Implements a Storm Spout that reads data from a given local file. + * Implements a Spout that reads data from a given local file. */ -public final class StormWordCountFileSpout extends StormFileSpout { +public final class WordCountFileSpout extends FileSpout { private static final long serialVersionUID = 2372251989250954503L; - public StormWordCountFileSpout(String path) { + public WordCountFileSpout(String path) { super(path); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java similarity index 77% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.java rename to flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java index 372f66fd54de9..eb96160ba3bc6 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java @@ -15,21 +15,21 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount.stormoperators; +package org.apache.flink.storm.wordcount.operators; import org.apache.flink.examples.java.wordcount.util.WordCountData; -import org.apache.flink.stormcompatibility.util.StormInMemorySpout; +import org.apache.flink.storm.util.InMemorySpout; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; /** - * Implements a Storm Spout that reads data from {@link WordCountData#WORDS}. + * Implements a Spout that reads data from {@link WordCountData#WORDS}. */ -public final class StormWordCountInMemorySpout extends StormInMemorySpout { +public final class WordCountInMemorySpout extends InMemorySpout { private static final long serialVersionUID = 8832143302409465843L; - public StormWordCountInMemorySpout() { + public WordCountInMemorySpout() { super(WordCountData.WORDS); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java similarity index 75% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java index a858f36b13ac9..781396c242f3f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.exclamation; +package org.apache.flink.storm.exclamation; -import org.apache.flink.stormcompatibility.api.StormTestBase; -import org.apache.flink.stormcompatibility.excamation.ExclamationWithStormBolt; -import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData; +import org.apache.flink.storm.excamation.ExclamationWithBolt; +import org.apache.flink.storm.exclamation.util.ExclamationData; +import org.apache.flink.storm.util.StormTestBase; import org.apache.flink.test.testdata.WordCountData; -public class ExclamationWithStormBoltITCase extends StormTestBase { +public class ExclamationWithBoltITCase extends StormTestBase { protected String textPath; protected String resultPath; @@ -43,7 +43,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - ExclamationWithStormBolt.main(new String[]{this.textPath, this.resultPath, this.exclamationNum}); + ExclamationWithBolt.main(new String[]{this.textPath, this.resultPath, this.exclamationNum}); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.java similarity index 75% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.java index 2a8ac24097691..36b8aed726605 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.exclamation; +package org.apache.flink.storm.exclamation; -import org.apache.flink.stormcompatibility.api.StormTestBase; -import org.apache.flink.stormcompatibility.excamation.ExclamationWithStormSpout; -import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData; +import org.apache.flink.storm.excamation.ExclamationWithSpout; +import org.apache.flink.storm.exclamation.util.ExclamationData; +import org.apache.flink.storm.util.StormTestBase; import org.apache.flink.test.testdata.WordCountData; -public class ExclamationWithStormSpoutITCase extends StormTestBase { +public class ExclamationWithSpoutITCase extends StormTestBase { protected String textPath; protected String resultPath; @@ -41,6 +41,6 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - ExclamationWithStormSpout.main(new String[]{this.textPath, this.resultPath}); + ExclamationWithSpout.main(new String[]{this.textPath, this.resultPath}); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/StormExclamationLocalITCase.java similarity index 79% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/StormExclamationLocalITCase.java index a19f3aff791ff..cec276fde1c71 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/StormExclamationLocalITCase.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.exclamation; +package org.apache.flink.storm.exclamation; -import org.apache.flink.stormcompatibility.api.StormTestBase; -import org.apache.flink.stormcompatibility.excamation.StormExclamationLocal; -import org.apache.flink.stormcompatibility.exclamation.util.ExclamationData; +import org.apache.flink.storm.excamation.ExclamationLocal; +import org.apache.flink.storm.exclamation.util.ExclamationData; +import org.apache.flink.storm.util.StormTestBase; import org.apache.flink.test.testdata.WordCountData; public class StormExclamationLocalITCase extends StormTestBase { @@ -43,6 +43,6 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - StormExclamationLocal.main(new String[]{this.textPath, this.resultPath, this.exclamationNum}); + ExclamationLocal.main(new String[]{this.textPath, this.resultPath, this.exclamationNum}); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/util/ExclamationData.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/util/ExclamationData.java similarity index 99% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/util/ExclamationData.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/util/ExclamationData.java index 8b823b5592977..3c435f982afda 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/util/ExclamationData.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/util/ExclamationData.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.exclamation.util; +package org.apache.flink.storm.exclamation.util; public class ExclamationData { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/BoltSplitITCase.java similarity index 88% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/BoltSplitITCase.java index 305245b14999a..dc174e7efb502 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/BoltSplitITCase.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; import org.junit.Test; @@ -22,7 +22,7 @@ public class BoltSplitITCase { @Test public void testTopology() throws Exception { - StormSplitStreamBoltLocal.main(new String[] { "0", "/dev/null" }); + SplitStreamBoltLocal.main(new String[] { "0", "/dev/null" }); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java index c40e0548cd6f9..c7b9c1d400008 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; import java.util.Map; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java similarity index 72% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java index 5f637d3d98161..d0973cb39ffdd 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; -import org.apache.flink.stormcompatibility.split.stormoperators.RandomSpout; -import org.apache.flink.stormcompatibility.split.stormoperators.VerifyAndEnrichBolt; -import org.apache.flink.stormcompatibility.util.OutputFormatter; -import org.apache.flink.stormcompatibility.util.StormBoltFileSink; -import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; -import org.apache.flink.stormcompatibility.util.TupleOutputFormatter; +import org.apache.flink.storm.api.FlinkTopologyBuilder; +import org.apache.flink.storm.split.operators.RandomSpout; +import org.apache.flink.storm.split.operators.VerifyAndEnrichBolt; +import org.apache.flink.storm.util.OutputFormatter; +import org.apache.flink.storm.util.BoltFileSink; +import org.apache.flink.storm.util.BoltPrintSink; +import org.apache.flink.storm.util.TupleOutputFormatter; public class SplitBoltTopology { public final static String spoutId = "randomSource"; @@ -48,11 +48,11 @@ public static FlinkTopologyBuilder buildTopology() { // read the text file from given input path final String[] tokens = outputPath.split(":"); final String outputFile = tokens[tokens.length - 1]; - builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)) - .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + builder.setBolt(sinkId, new BoltFileSink(outputFile, formatter)) + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); } else { - builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4) - .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); + builder.setBolt(sinkId, new BoltPrintSink(formatter), 4) + .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); } return builder; @@ -73,12 +73,12 @@ static boolean parseParameters(final String[] args) { seed = Long.parseLong(args[0]); outputPath = args[1]; } else { - System.err.println("Usage: SplitBoltTopology "); + System.err.println("Usage: SplitStreamBoltLocal "); return false; } } else { System.out.println("Executing SplitBoltTopology example with random data"); - System.out.println(" Usage: SplitBoltTopology "); + System.out.println(" Usage: SplitStreamBoltLocal "); } return true; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java similarity index 75% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java index 613fd10b1f0bd..2b3b6a804fd94 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; -import org.apache.flink.stormcompatibility.split.stormoperators.RandomSpout; -import org.apache.flink.stormcompatibility.split.stormoperators.VerifyAndEnrichBolt; -import org.apache.flink.stormcompatibility.util.OutputFormatter; -import org.apache.flink.stormcompatibility.util.StormBoltFileSink; -import org.apache.flink.stormcompatibility.util.StormBoltPrintSink; -import org.apache.flink.stormcompatibility.util.TupleOutputFormatter; +import org.apache.flink.storm.api.FlinkTopologyBuilder; +import org.apache.flink.storm.split.operators.RandomSpout; +import org.apache.flink.storm.split.operators.VerifyAndEnrichBolt; +import org.apache.flink.storm.util.OutputFormatter; +import org.apache.flink.storm.util.BoltFileSink; +import org.apache.flink.storm.util.BoltPrintSink; +import org.apache.flink.storm.util.TupleOutputFormatter; public class SplitSpoutTopology { public final static String spoutId = "randomSplitSource"; @@ -46,10 +46,10 @@ public static FlinkTopologyBuilder buildTopology() { // read the text file from given input path final String[] tokens = outputPath.split(":"); final String outputFile = tokens[tokens.length - 1]; - builder.setBolt(sinkId, new StormBoltFileSink(outputFile, formatter)) + builder.setBolt(sinkId, new BoltFileSink(outputFile, formatter)) .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); } else { - builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4) + builder.setBolt(sinkId, new BoltPrintSink(formatter), 4) .shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId); } @@ -71,12 +71,12 @@ static boolean parseParameters(final String[] args) { seed = Long.parseLong(args[0]); outputPath = args[1]; } else { - System.err.println("Usage: SplitSpoutTopology "); + System.err.println("Usage: SplitStreamSpoutLocal "); return false; } } else { System.out.println("Executing SplitSpoutTopology example with random data"); - System.out.println(" Usage: SplitSpoutTopology "); + System.out.println(" Usage: SplitStreamSpoutLocal "); } return true; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java similarity index 87% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java index 028f6d113dff1..e2c22f9f1ca6e 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; -import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.storm.api.FlinkLocalCluster; +import org.apache.flink.storm.api.FlinkTopologyBuilder; import backtype.storm.utils.Utils; -public class StormSplitStreamBoltLocal { +public class SplitStreamBoltLocal { public final static String topologyId = "Bolt split stream example"; // ************************************************************************* diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java similarity index 87% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java index cc5acd969e448..2070f6624665e 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; -import org.apache.flink.stormcompatibility.api.FlinkLocalCluster; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.storm.api.FlinkLocalCluster; +import org.apache.flink.storm.api.FlinkTopologyBuilder; import backtype.storm.utils.Utils; -public class StormSplitStreamSpoutLocal { +public class SplitStreamSpoutLocal { public final static String topologyId = "Spout split stream example"; // ************************************************************************* diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SpoutSplitITCase.java similarity index 88% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SpoutSplitITCase.java index f30e160c51f49..8e0fda9c1c811 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SpoutSplitITCase.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.split; +package org.apache.flink.storm.split; import org.junit.Test; @@ -22,7 +22,7 @@ public class SpoutSplitITCase { @Test public void testTopology() throws Exception { - StormSplitStreamSpoutLocal.main(new String[] { "0", "/dev/null" }); + SplitStreamSpoutLocal.main(new String[] { "0", "/dev/null" }); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/util/StormTestBase.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/util/StormTestBase.java index dd6d0d9266418..32dac7b9946f5 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/util/StormTestBase.java @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.util; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.util.AbstractTestBase; - import org.junit.Test; import static org.junit.Assert.fail; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountITCase.java similarity index 90% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountITCase.java index c9516ff81cd6b..62d23ab0f193e 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountITCase.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; -import org.apache.flink.stormcompatibility.api.StormTestBase; +import org.apache.flink.storm.util.StormTestBase; +import org.apache.flink.storm.wordcount.BoltTokenizerWordCount; import org.apache.flink.test.testdata.WordCountData; public class BoltTokenizerWordCountITCase extends StormTestBase { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojoITCase.java similarity index 90% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojoITCase.java index 351014ec55fc3..009bdc213414a 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojoITCase.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; -import org.apache.flink.stormcompatibility.api.StormTestBase; +import org.apache.flink.storm.util.StormTestBase; +import org.apache.flink.storm.wordcount.BoltTokenizerWordCountPojo; import org.apache.flink.test.testdata.WordCountData; public class BoltTokenizerWordCountPojoITCase extends StormTestBase { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNamesITCase.java similarity index 90% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNamesITCase.java index c2ed0884ce282..321015bbd84d4 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNamesITCase.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; -import org.apache.flink.stormcompatibility.api.StormTestBase; +import org.apache.flink.storm.util.StormTestBase; +import org.apache.flink.storm.wordcount.BoltTokenizerWordCountWithNames; import org.apache.flink.test.testdata.WordCountData; public class BoltTokenizerWordCountWithNamesITCase extends StormTestBase { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/SpoutSourceWordCountITCase.java similarity index 90% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/SpoutSourceWordCountITCase.java index 93361c52e0dd8..0cff21184a4fe 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/SpoutSourceWordCountITCase.java @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; -import org.apache.flink.stormcompatibility.api.StormTestBase; +import org.apache.flink.storm.util.StormTestBase; +import org.apache.flink.storm.wordcount.SpoutSourceWordCount; import org.apache.flink.test.testdata.WordCountData; public class SpoutSourceWordCountITCase extends StormTestBase { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.java similarity index 82% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.java index a9e98849071f1..39e7a25598a03 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.java @@ -16,12 +16,13 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; -import org.apache.flink.stormcompatibility.api.StormTestBase; +import org.apache.flink.storm.util.StormTestBase; +import org.apache.flink.storm.wordcount.WordCountLocal; import org.apache.flink.test.testdata.WordCountData; -public class StormWordCountLocalNamedITCase extends StormTestBase { +public class WordCountLocalITCase extends StormTestBase { protected String textPath; protected String resultPath; @@ -39,7 +40,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - StormWordCountNamedLocal.main(new String[] { this.textPath, this.resultPath }); + WordCountLocal.main(new String[]{this.textPath, this.resultPath}); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.java similarity index 81% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java rename to flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.java index 6b51cbd598a5c..78acfe5c3cccb 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.java @@ -16,12 +16,13 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wordcount; +package org.apache.flink.storm.wordcount; -import org.apache.flink.stormcompatibility.api.StormTestBase; +import org.apache.flink.storm.util.StormTestBase; +import org.apache.flink.storm.wordcount.WordCountLocalByName; import org.apache.flink.test.testdata.WordCountData; -public class StormWordCountLocalITCase extends StormTestBase { +public class WordCountLocalNamedITCase extends StormTestBase { protected String textPath; protected String resultPath; @@ -39,7 +40,7 @@ protected void postSubmit() throws Exception { @Override protected void testProgram() throws Exception { - StormWordCountLocal.main(new String[]{this.textPath, this.resultPath}); + WordCountLocalByName.main(new String[] { this.textPath, this.resultPath }); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j-test.properties b/flink-contrib/flink-storm-examples/src/test/resources/log4j-test.properties similarity index 100% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j-test.properties rename to flink-contrib/flink-storm-examples/src/test/resources/log4j-test.properties diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j.properties b/flink-contrib/flink-storm-examples/src/test/resources/log4j.properties similarity index 100% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j.properties rename to flink-contrib/flink-storm-examples/src/test/resources/log4j.properties diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/logback-test.xml b/flink-contrib/flink-storm-examples/src/test/resources/logback-test.xml similarity index 100% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/logback-test.xml rename to flink-contrib/flink-storm-examples/src/test/resources/logback-test.xml diff --git a/flink-contrib/flink-storm/README.md b/flink-contrib/flink-storm/README.md new file mode 100644 index 0000000000000..239780c1bdceb --- /dev/null +++ b/flink-contrib/flink-storm/README.md @@ -0,0 +1,15 @@ +# flink-storm + +`flink-storm` is compatibility layer for Apache Storm and allows to embed Spouts or Bolts unmodified within a regular Flink streaming program (`SpoutWrapper` and `BoltWrapper`). +Additionally, a whole Storm topology can be submitted to Flink (see `FlinkTopologyBuilder`, `FlinkLocalCluster`, and `FlinkSubmitter`). +Only a few minor changes to the original submitting code are required. +The code that builds the topology itself, can be reused unmodified. See `flink-storm-examples` for a simple word-count example. + +The following Storm features are not (yet/fully) supported by the compatibility layer right now: +* tuple meta information +* no fault-tolerance guarantees (ie, calls to `ack()`/`fail()` and anchoring is ignored) +* for whole Storm topologies the following is not supported by Flink: + * direct emit connection pattern + * activating/deactivating and rebalancing of topologies + * task hooks + * metrics diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/pom.xml b/flink-contrib/flink-storm/pom.xml similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/pom.xml rename to flink-contrib/flink-storm/pom.xml index cced6784f8baa..657b974cd3697 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/pom.xml +++ b/flink-contrib/flink-storm/pom.xml @@ -24,13 +24,13 @@ under the License. org.apache.flink - flink-storm-compatibility-parent + flink-contrib-parent 0.10-SNAPSHOT .. - flink-storm-compatibility-core - flink-storm-compatibility-core + flink-storm + flink-storm jar diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java similarity index 99% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java index 4676102c6e159..5f0ee2126b8a9 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; import akka.actor.ActorRef; import akka.actor.ActorSystem; @@ -48,7 +48,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus; -import org.apache.flink.stormcompatibility.util.StormConfig; +import org.apache.flink.storm.util.StormConfig; import scala.Some; import scala.concurrent.Await; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java index 9b3fb543afb6a..868801b06dead 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; import backtype.storm.LocalCluster; import backtype.storm.generated.ClusterSummary; @@ -31,8 +31,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.minicluster.FlinkMiniCluster; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; -import org.apache.flink.stormcompatibility.util.StormConfig; - +import org.apache.flink.storm.util.StormConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java similarity index 92% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java index 3eee8d6402956..88d2dfe58e4a2 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.api; import backtype.storm.topology.OutputFieldsDeclarer; import backtype.storm.tuple.Fields; @@ -34,10 +34,10 @@ *
    * CAUTION: Flink does not support direct emit. */ -public final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer { +final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer { /** The declared output streams and schemas. */ - public final HashMap outputStreams = new HashMap(); + final HashMap outputStreams = new HashMap(); @Override public void declare(final Fields fields) { @@ -91,7 +91,7 @@ public void declareStream(final String streamId, final boolean direct, final Fie * @throws IllegalArgumentException * If no output schema was declared for the specified stream or if more then 25 attributes got declared. */ - public TypeInformation getOutputType(final String streamId) throws IllegalArgumentException { + TypeInformation getOutputType(final String streamId) throws IllegalArgumentException { if (streamId == null) { return null; } @@ -155,7 +155,7 @@ public int compareTo(final DefaultComparable o) { * @return array of {@code int}s that contains the index within the output schema for each attribute in the given * list */ - public int[] getGroupingFieldIndexes(final String streamId, final List groupingFields) { + int[] getGroupingFieldIndexes(final String streamId, final List groupingFields) { final int[] fieldIndexes = new int[groupingFields.size()]; for (int i = 0; i < fieldIndexes.length; ++i) { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java similarity index 99% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java index 5f3f31ebc2d81..9b03c68376ed8 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; import backtype.storm.Config; import backtype.storm.StormSubmitter; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java similarity index 98% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java index 179466ed66895..531d6dfc85b7a 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; import backtype.storm.generated.StormTopology; import org.apache.flink.api.common.JobExecutionResult; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java similarity index 89% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java index d62d56b8d83a8..99de0e2a7c2ff 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java @@ -16,7 +16,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; import backtype.storm.generated.ComponentCommon; import backtype.storm.generated.GlobalStreamId; @@ -34,15 +34,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.stormcompatibility.util.FiniteStormSpout; -import org.apache.flink.stormcompatibility.util.FlinkOutputFieldsDeclarer; -import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector; -import org.apache.flink.stormcompatibility.util.SplitStreamType; -import org.apache.flink.stormcompatibility.util.SplitStreamTypeKeySelector; -import org.apache.flink.stormcompatibility.wrappers.AbstractStormSpoutWrapper; -import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; -import org.apache.flink.stormcompatibility.wrappers.StormBoltWrapper; -import org.apache.flink.stormcompatibility.wrappers.StormSpoutWrapper; +import org.apache.flink.storm.util.SplitStreamType; +import org.apache.flink.storm.util.SplitStreamTypeKeySelector; +import org.apache.flink.storm.util.StormStreamSelector; +import org.apache.flink.storm.wrappers.BoltWrapper; +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; @@ -100,17 +96,11 @@ public FlinkTopology createTopology() { this.outputStreams.put(spoutId, sourceStreams); declarers.put(spoutId, declarer); - AbstractStormSpoutWrapper spoutWrapper; - - if (userSpout instanceof FiniteStormSpout) { - spoutWrapper = new FiniteStormSpoutWrapper((FiniteStormSpout) userSpout); - } else { - spoutWrapper = new StormSpoutWrapper(userSpout); - } + final SpoutWrapper spoutWrapper = new SpoutWrapper(userSpout); spoutWrapper.setStormTopology(stormTopology); DataStreamSource source; - HashMap outputStreams = new HashMap(); + final HashMap outputStreams = new HashMap(); if (sourceStreams.size() == 1) { final String outputStreamId = (String) sourceStreams.keySet().toArray()[0]; source = env.addSource(spoutWrapper, spoutId, @@ -119,7 +109,7 @@ public FlinkTopology createTopology() { } else { source = env.addSource(spoutWrapper, spoutId, TypeExtractor.getForClass(SplitStreamType.class)); - SplitStream splitSource = source.split(new FlinkStormStreamSelector()); + SplitStream splitSource = source.split(new StormStreamSelector()); for (String streamId : sourceStreams.keySet()) { outputStreams.put(streamId, splitSource.select(streamId)); @@ -181,7 +171,7 @@ public FlinkTopology createTopology() { final String producerId = stormInputStream.getKey().get_componentId(); final String inputStreamId = stormInputStream.getKey().get_streamId(); - HashMap producer = availableInputs.get(producerId); + final HashMap producer = availableInputs.get(producerId); if (producer != null) { makeProgress = true; @@ -224,8 +214,8 @@ public FlinkTopology createTopology() { "Flink only supports (local-or-)shuffle, fields, all, and global grouping"); } - SingleOutputStreamOperator outputStream; - StormBoltWrapper boltWrapper; + final SingleOutputStreamOperator outputStream; + final BoltWrapper boltWrapper; if (boltOutputStreams.size() < 2) { // single output stream or sink String outputStreamId = null; if (boltOutputStreams.size() == 1) { @@ -234,13 +224,13 @@ public FlinkTopology createTopology() { final TypeInformation outType = declarer .getOutputType(outputStreamId); - boltWrapper = new StormBoltWrapper(userBolt, this.outputStreams + boltWrapper = new BoltWrapper(userBolt, this.outputStreams .get(producerId).get(inputStreamId)); outputStream = inputStream.transform(boltId, outType, boltWrapper); if (outType != null) { // only for non-sink nodes - HashMap op = new HashMap(); + final HashMap op = new HashMap(); op.put(outputStreamId, outputStream); availableInputs.put(boltId, op); } @@ -248,13 +238,13 @@ public FlinkTopology createTopology() { final TypeInformation outType = TypeExtractor .getForClass(SplitStreamType.class); - boltWrapper = new StormBoltWrapper(userBolt, this.outputStreams.get(producerId).get(inputStreamId)); + boltWrapper = new BoltWrapper(userBolt, this.outputStreams.get(producerId).get(inputStreamId)); outputStream = inputStream.transform(boltId, outType, boltWrapper); - SplitStream splitStreams = outputStream - .split(new FlinkStormStreamSelector()); + final SplitStream splitStreams = outputStream + .split(new StormStreamSelector()); - HashMap op = new HashMap(); + final HashMap op = new HashMap(); for (String outputStreamId : boltOutputStreams.keySet()) { op.put(outputStreamId, splitStreams.select(outputStreamId)); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java similarity index 66% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java index 114fa7c0d97e3..99c258387c161 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java @@ -15,19 +15,16 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; - -import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper; +package org.apache.flink.storm.util; import backtype.storm.topology.IRichSpout; /** - * This interface represents a Storm spout that emits a finite number of records. Common Storm - * spouts emit infinite streams by default. To change this behaviour and take advantage of - * Flink's finite-source capabilities, the spout should implement this interface. To wrap - * {@link FiniteStormSpout} separately, use {@link FiniteStormSpoutWrapper}. + * This interface represents a spout that emits a finite number of records. Common spouts emit infinite streams by + * default. To change this behavior and take advantage of Flink's finite-source capabilities, the spout should implement + * this interface. */ -public interface FiniteStormSpout extends IRichSpout { +public interface FiniteSpout extends IRichSpout { /** * When returns true, the spout has reached the end of the stream. diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamMapper.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamMapper.java index 9cb44ec53c58b..1fb5e024e175c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamMapper.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.streaming.api.datastream.DataStream; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java similarity index 89% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java index 9c7e47759de70..a4b5f8ee48329 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import org.apache.flink.streaming.api.datastream.DataStream; /** - * Used by {@link org.apache.flink.stormcompatibility.wrappers.AbstractStormCollector AbstractStormCollector} to wrap + * Used by {@link org.apache.flink.storm.wrappers.AbstractStormCollector AbstractStormCollector} to wrap * output tuples if multiple output streams are declared. For this case, the Flink output data stream must be split via * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector) .split(...)} using - * {@link FlinkStormStreamSelector}. + * {@link StormStreamSelector}. */ public class SplitStreamType { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java similarity index 92% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java index 200f77241a965..44c693c23f73d 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple; @@ -24,7 +24,7 @@ /** * {@link SplitStreamTypeKeySelector} is a specific grouping key selector for streams that are selected via - * {@link FlinkStormStreamSelector} from a Spout or Bolt that declares multiple output streams. + * {@link StormStreamSelector} from a Spout or Bolt that declares multiple output streams. * * It extracts the wrapped {@link Tuple} type from the {@link SplitStreamType} tuples and applies a regular * {@link ArrayKeySelector} on it. diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/StormConfig.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java similarity index 94% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/StormConfig.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java index 6726ae87fd23c..6550990b6ce69 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/StormConfig.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import java.util.Collection; import java.util.HashMap; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java similarity index 88% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java index 7e60a87560ced..d9f4178a1008d 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java @@ -15,19 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import java.util.ArrayList; import java.util.HashMap; import java.util.List; -import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder; +import org.apache.flink.storm.api.FlinkTopologyBuilder; import org.apache.flink.streaming.api.collector.selector.OutputSelector; /** * Used by {@link FlinkTopologyBuilder} to split multiple declared output streams within Flink. */ -public final class FlinkStormStreamSelector implements OutputSelector> { +public final class StormStreamSelector implements OutputSelector> { private static final long serialVersionUID = 2553423379715401023L; /** internal cache to avoid short living ArrayList objects. */ diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/AbstractStormCollector.java similarity index 96% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/AbstractStormCollector.java index 7b35a64dd117a..85d895c3240a2 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/AbstractStormCollector.java @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple25; -import org.apache.flink.stormcompatibility.util.SplitStreamType; +import org.apache.flink.storm.util.SplitStreamType; import java.util.HashMap; import java.util.List; @@ -54,7 +54,7 @@ abstract class AbstractStormCollector { * @throws UnsupportedOperationException * if the specified number of attributes is greater than 25 */ - public AbstractStormCollector(final HashMap numberOfAttributes) + AbstractStormCollector(final HashMap numberOfAttributes) throws UnsupportedOperationException { assert (numberOfAttributes != null); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java similarity index 77% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java index e810214fe92e5..58fd0988d9e3c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import backtype.storm.task.IOutputCollector; import backtype.storm.tuple.Tuple; @@ -30,17 +30,17 @@ import java.util.List; /** - * A {@link StormBoltCollector} is used by {@link StormBoltWrapper} to provided an Storm compatible + * A {@link BoltCollector} is used by {@link BoltWrapper} to provided an Storm compatible * output collector to the wrapped bolt. It transforms the emitted Storm tuples into Flink tuples * and emits them via the provide {@link Output} object. */ -class StormBoltCollector extends AbstractStormCollector implements IOutputCollector { +class BoltCollector extends AbstractStormCollector implements IOutputCollector { /** The Flink output Collector */ private final Collector flinkOutput; /** - * Instantiates a new {@link StormBoltCollector} that emits Flink tuples to the given Flink output object. If the + * Instantiates a new {@link BoltCollector} that emits Flink tuples to the given Flink output object. If the * number of attributes is negative, any output type is supported (ie, raw type). If the number of attributes is * between 0 and 25, the output type is {@link Tuple0} to {@link Tuple25}, respectively. * @@ -51,7 +51,7 @@ class StormBoltCollector extends AbstractStormCollector implements IOu * @throws UnsupportedOperationException * if the specified number of attributes is greater than 25 */ - public StormBoltCollector(final HashMap numberOfAttributes, + BoltCollector(final HashMap numberOfAttributes, final Collector flinkOutput) throws UnsupportedOperationException { super(numberOfAttributes); assert (flinkOutput != null); @@ -68,7 +68,6 @@ protected List doEmit(final OUT flinkTuple) { @Override public void reportError(final Throwable error) { // not sure, if Flink can support this - throw new UnsupportedOperationException("Not implemented yet"); } @Override @@ -82,13 +81,9 @@ public void emitDirect(final int taskId, final String streamId, final Collection } @Override - public void ack(final Tuple input) { - throw new UnsupportedOperationException("Currently, acking/failing is not supported by Flink"); - } + public void ack(final Tuple input) {} @Override - public void fail(final Tuple input) { - throw new UnsupportedOperationException("Currently, acking/failing is not supported by Flink"); - } + public void fail(final Tuple input) {} } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java similarity index 83% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java index 715d6df814b99..b16fc097150b4 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import java.util.Collection; import java.util.HashMap; @@ -30,8 +30,8 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.configuration.Configuration; -import org.apache.flink.stormcompatibility.util.SplitStreamType; -import org.apache.flink.stormcompatibility.util.StormConfig; +import org.apache.flink.storm.util.SplitStreamType; +import org.apache.flink.storm.util.StormConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.TimestampedCollector; @@ -41,7 +41,7 @@ import com.google.common.collect.Sets; /** - * A {@link StormBoltWrapper} wraps an {@link IRichBolt} in order to execute the Storm bolt within a Flink Streaming + * A {@link BoltWrapper} wraps an {@link IRichBolt} in order to execute the Storm bolt within a Flink Streaming * program. It takes the Flink input tuples of type {@code IN} and transforms them into {@link StormTuple}s that the * bolt can process. Furthermore, it takes the bolt's output tuples and transforms them into Flink tuples of type * {@code OUT} (see {@link AbstractStormCollector} for supported types).
    @@ -51,7 +51,7 @@ * Furthermore, acking and failing of tuples as well as accessing tuple attributes by field names is not supported so * far. */ -public class StormBoltWrapper extends AbstractStreamOperator implements OneInputStreamOperator { +public class BoltWrapper extends AbstractStreamOperator implements OneInputStreamOperator { private static final long serialVersionUID = -4788589118464155835L; /** The wrapped Storm {@link IRichBolt bolt}. */ @@ -62,7 +62,6 @@ public class StormBoltWrapper extends AbstractStreamOperator imple private final Fields inputSchema; /** The original Storm topology. */ protected StormTopology stormTopology; - /** * We have to use this because Operators must output * {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}. @@ -70,7 +69,7 @@ public class StormBoltWrapper extends AbstractStreamOperator imple private TimestampedCollector flinkCollector; /** - * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * Instantiates a new {@link BoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. As no input schema is defined, attribute-by-name access in only possible * for POJO input types. The output type will be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's * declared number of attributes. @@ -80,12 +79,12 @@ public class StormBoltWrapper extends AbstractStreamOperator imple * @throws IllegalArgumentException * If the number of declared output attributes is not with range [0;25]. */ - public StormBoltWrapper(final IRichBolt bolt) throws IllegalArgumentException { + public BoltWrapper(final IRichBolt bolt) throws IllegalArgumentException { this(bolt, null, (Collection) null); } /** - * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * Instantiates a new {@link BoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. The given input schema enable attribute-by-name access for input types * {@link Tuple0} to {@link Tuple25}. The output type will be one of {@link Tuple0} to {@link Tuple25} depending on * the bolt's declared number of attributes. @@ -97,13 +96,13 @@ public StormBoltWrapper(final IRichBolt bolt) throws IllegalArgumentException { * @throws IllegalArgumentException * If the number of declared output attributes is not with range [0;25]. */ - public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema) + public BoltWrapper(final IRichBolt bolt, final Fields inputSchema) throws IllegalArgumentException { this(bolt, inputSchema, (Collection) null); } /** - * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * Instantiates a new {@link BoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. As no input schema is defined, attribute-by-name access in only possible * for POJO input types. The output type can be any type if parameter {@code rawOutput} is {@code true} and the * bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one @@ -119,13 +118,13 @@ public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema) * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range * [1;25]. */ - public StormBoltWrapper(final IRichBolt bolt, final String[] rawOutputs) + public BoltWrapper(final IRichBolt bolt, final String[] rawOutputs) throws IllegalArgumentException { this(bolt, null, Sets.newHashSet(rawOutputs)); } /** - * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * Instantiates a new {@link BoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. As no input schema is defined, attribute-by-name access in only possible * for POJO input types. The output type can be any type if parameter {@code rawOutput} is {@code true} and the * bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one @@ -141,13 +140,13 @@ public StormBoltWrapper(final IRichBolt bolt, final String[] rawOutputs) * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range * [1;25]. */ - public StormBoltWrapper(final IRichBolt bolt, final Collection rawOutputs) + public BoltWrapper(final IRichBolt bolt, final Collection rawOutputs) throws IllegalArgumentException { this(bolt, null, rawOutputs); } /** - * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * Instantiates a new {@link BoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. The given input schema enable attribute-by-name access for input types * {@link Tuple0} to {@link Tuple25}. The output type can be any type if parameter {@code rawOutput} is {@code true} * and the bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will @@ -165,13 +164,13 @@ public StormBoltWrapper(final IRichBolt bolt, final Collection rawOutput * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range * [0;25]. */ - public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema, + public BoltWrapper(final IRichBolt bolt, final Fields inputSchema, final String[] rawOutputs) throws IllegalArgumentException { this(bolt, inputSchema, Sets.newHashSet(rawOutputs)); } /** - * Instantiates a new {@link StormBoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be + * Instantiates a new {@link BoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be * used within a Flink streaming program. The given input schema enable attribute-by-name access for input types * {@link Tuple0} to {@link Tuple25}. The output type can be any type if parameter {@code rawOutput} is {@code true} * and the bolt's number of declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will @@ -189,11 +188,11 @@ public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema, * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range * [0;25]. */ - public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema, + public BoltWrapper(final IRichBolt bolt, final Fields inputSchema, final Collection rawOutputs) throws IllegalArgumentException { this.bolt = bolt; this.inputSchema = inputSchema; - this.numberOfAttributes = StormWrapperSetupHelper.getNumberOfAttributes(bolt, rawOutputs); + this.numberOfAttributes = WrapperSetupHelper.getNumberOfAttributes(bolt, rawOutputs); } /** @@ -210,13 +209,11 @@ public void setStormTopology(StormTopology stormTopology) { public void open(final Configuration parameters) throws Exception { super.open(parameters); - final TopologyContext topologyContext = StormWrapperSetupHelper.createTopologyContext( - super.runtimeContext, this.bolt, this.stormTopology, null); - flinkCollector = new TimestampedCollector(output); + this.flinkCollector = new TimestampedCollector(output); OutputCollector stormCollector = null; if (this.numberOfAttributes.size() > 0) { - stormCollector = new OutputCollector(new StormBoltCollector( + stormCollector = new OutputCollector(new BoltCollector( this.numberOfAttributes, flinkCollector)); } @@ -231,6 +228,9 @@ public void open(final Configuration parameters) throws Exception { } } + final TopologyContext topologyContext = WrapperSetupHelper.createTopologyContext( + super.runtimeContext, this.bolt, this.stormTopology, stormConfig); + this.bolt.prepare(stormConfig, topologyContext, stormCollector); } @@ -242,7 +242,7 @@ public void dispose() { @SuppressWarnings("unchecked") @Override public void processElement(final StreamRecord element) throws Exception { - flinkCollector.setTimestamp(element.getTimestamp()); + this.flinkCollector.setTimestamp(element.getTimestamp()); IN value = element.getValue(); if (value instanceof SplitStreamType) { this.bolt.execute(new StormTuple(((SplitStreamType) value).value, @@ -254,7 +254,7 @@ public void processElement(final StreamRecord element) throws Exception { @Override public void processWatermark(Watermark mark) throws Exception { - output.emitWatermark(mark); + this.output.emitWatermark(mark); } } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java index 14af8305e721f..68368bf531bdb 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.wrappers; import backtype.storm.generated.StormTopology; import backtype.storm.hooks.ITaskHook; @@ -38,13 +38,14 @@ * {@link FlinkTopologyContext} is a {@link TopologyContext} that overwrites certain method that are not applicable when * a Storm topology is executed within Flink. */ -public final class FlinkTopologyContext extends TopologyContext { +final class FlinkTopologyContext extends TopologyContext { /** * Instantiates a new {@link FlinkTopologyContext} for a given Storm topology. The context object is instantiated * for each parallel task */ - public FlinkTopologyContext(final StormTopology topology, @SuppressWarnings("rawtypes") final Map stormConf, + FlinkTopologyContext(final StormTopology topology, + @SuppressWarnings("rawtypes") final Map stormConf, final Map taskToComponent, final Map> componentToSortedTasks, final Map> componentToStreamToFields, final String stormId, final String codeDir, final String pidDir, final Integer taskId, final Integer workerPort, final List workerTasks, diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java similarity index 86% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java index 3cd27d4fb4b10..507305b00ba28 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import java.util.HashMap; @@ -24,12 +24,14 @@ import backtype.storm.utils.Utils; /** - * {@link SetupOutputFieldsDeclarer} is used by {@link StormWrapperSetupHelper} to determine the output streams and + * {@link SetupOutputFieldsDeclarer} is used by {@link WrapperSetupHelper} to determine the output streams and * number of attributes declared by the wrapped spout's or bolt's {@code declare(...)}/{@code declareStream(...)} * method. */ class SetupOutputFieldsDeclarer implements OutputFieldsDeclarer { + /** The declared output streams and schemas. */ + HashMap outputStreams = new HashMap(); /** The number of attributes for each declared stream by the wrapped operator. */ HashMap outputSchemas = new HashMap(); @@ -57,6 +59,7 @@ public void declareStream(final String streamId, final boolean direct, final Fie throw new UnsupportedOperationException("Direct emit is not supported by Flink"); } + this.outputStreams.put(streamId, fields); this.outputSchemas.put(streamId, fields.size()); } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java similarity index 83% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java index 5a20056ae8474..91fc09081314f 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import backtype.storm.spout.ISpoutOutputCollector; @@ -27,17 +27,17 @@ import java.util.List; /** - * A {@link StormSpoutCollector} is used by {@link AbstractStormSpoutWrapper} to provided an Storm + * A {@link SpoutCollector} is used by {@link AbstractStormSpoutWrapper} to provided an Storm * compatible output collector to the wrapped spout. It transforms the emitted Storm tuples into * Flink tuples and emits them via the provide {@link SourceContext} object. */ -class StormSpoutCollector extends AbstractStormCollector implements ISpoutOutputCollector { +class SpoutCollector extends AbstractStormCollector implements ISpoutOutputCollector { /** The Flink source context object */ private final SourceContext flinkContext; /** - * Instantiates a new {@link StormSpoutCollector} that emits Flink tuples to the given Flink source context. If the + * Instantiates a new {@link SpoutCollector} that emits Flink tuples to the given Flink source context. If the * number of attributes is specified as zero, any output type is supported. If the number of attributes is between 0 * to 25, the output type is {@link Tuple0} to {@link Tuple25}, respectively. * @@ -48,7 +48,7 @@ class StormSpoutCollector extends AbstractStormCollector implements IS * @throws UnsupportedOperationException * if the specified number of attributes is greater than 25 */ - public StormSpoutCollector(final HashMap numberOfAttributes, + SpoutCollector(final HashMap numberOfAttributes, final SourceContext flinkContext) throws UnsupportedOperationException { super(numberOfAttributes); assert (flinkContext != null); @@ -65,7 +65,6 @@ protected List doEmit(final OUT flinkTuple) { @Override public void reportError(final Throwable error) { // not sure, if Flink can support this - throw new UnsupportedOperationException("Not implemented yet"); } @Override diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java new file mode 100644 index 0000000000000..914a19d35901e --- /dev/null +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java @@ -0,0 +1,274 @@ +/* + * 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.storm.wrappers; + +import java.util.Collection; +import java.util.HashMap; + +import backtype.storm.generated.StormTopology; +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.topology.IRichSpout; + +import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; +import org.apache.flink.api.java.tuple.Tuple0; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple25; +import org.apache.flink.storm.util.FiniteSpout; +import org.apache.flink.storm.util.StormConfig; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; + +import com.google.common.collect.Sets; + +/** + * A {@link SpoutWrapper} wraps an {@link IRichSpout} in order to execute it within a Flink Streaming program. It + * takes the spout's output tuples and transforms them into Flink tuples of type {@code OUT} (see + * {@link SpoutCollector} for supported types).
    + *
    + * Per default, {@link SpoutWrapper} calls the wrapped spout's {@link IRichSpout#nextTuple() nextTuple()} method in + * an infinite loop.
    + * Alternatively, {@link SpoutWrapper} can call {@link IRichSpout#nextTuple() nextTuple()} for a finite number of + * times and terminate automatically afterwards (for finite input streams). The number of {@code nextTuple()} calls can + * be specified as a certain number of invocations or can be undefined. In the undefined case, {@link SpoutWrapper} + * terminates if no record was emitted to the output collector for the first time during a call to + * {@link IRichSpout#nextTuple() nextTuple()}.
    + * If the given spout implements {@link FiniteSpout} interface and {@link #numberOfInvocations} is not provided or + * is {@code null}, {@link SpoutWrapper} calls {@link IRichSpout#nextTuple() nextTuple()} method until + * {@link FiniteSpout#reachedEnd()} returns true. + */ +public final class SpoutWrapper extends RichParallelSourceFunction { + private static final long serialVersionUID = -218340336648247605L; + + /** Number of attributes of the spouts's output tuples per stream. */ + private final HashMap numberOfAttributes; + /** The wrapped {@link IRichSpout spout}. */ + private final IRichSpout spout; + /** The wrapper of the given Flink collector. */ + private SpoutCollector collector; + /** Indicates, if the source is still running or was canceled. */ + private volatile boolean isRunning = true; + /** The number of {@link IRichSpout#nextTuple()} calls. */ + private Integer numberOfInvocations; // do not use int -> null indicates an infinite loop + /** The original Storm topology. */ + private StormTopology stormTopology; + + /** + * Instantiates a new {@link SpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} method of + * the given {@link IRichSpout spout} in an infinite loop. The output type will be one of {@link Tuple0} to + * {@link Tuple25} depending on the spout's declared number of attributes. + * + * @param spout + * The {@link IRichSpout spout} to be used. + * @throws IllegalArgumentException + * If the number of declared output attributes is not with range [0;25]. + */ + public SpoutWrapper(final IRichSpout spout) throws IllegalArgumentException { + this(spout, (Collection) null, null); + } + + /** + * Instantiates a new {@link SpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} method of + * the given {@link IRichSpout spout} a finite number of times. The output type will be one of {@link Tuple0} to + * {@link Tuple25} depending on the spout's declared number of attributes. + * + * @param spout + * The {@link IRichSpout spout} to be used. + * @param numberOfInvocations + * The number of calls to {@link IRichSpout#nextTuple()}. If value is negative, {@link SpoutWrapper} + * terminates if no tuple was emitted for the first time. If value is {@code null}, finite invocation is + * disabled. + * @throws IllegalArgumentException + * If the number of declared output attributes is not with range [0;25]. + */ + public SpoutWrapper(final IRichSpout spout, final Integer numberOfInvocations) + throws IllegalArgumentException { + this(spout, (Collection) null, numberOfInvocations); + } + + /** + * Instantiates a new {@link SpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} method of + * the given {@link IRichSpout spout} in an infinite loop. The output type can be any type if parameter + * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is + * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared + * number of attributes. + * + * @param spout + * The {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. (Can be {@code null}.) + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public SpoutWrapper(final IRichSpout spout, final String[] rawOutputs) + throws IllegalArgumentException { + this(spout, Sets.newHashSet(rawOutputs), null); + } + + /** + * Instantiates a new {@link SpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} method of + * the given {@link IRichSpout spout} a finite number of times. The output type can be any type if parameter + * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is + * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared + * number of attributes. + * + * @param spout + * The {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. (Can be {@code null}.) + * @param numberOfInvocations + * The number of calls to {@link IRichSpout#nextTuple()}. If value is negative, {@link SpoutWrapper} + * terminates if no tuple was emitted for the first time. If value is {@code null}, finite invocation is + * disabled. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public SpoutWrapper(final IRichSpout spout, final String[] rawOutputs, + final Integer numberOfInvocations) throws IllegalArgumentException { + this(spout, Sets.newHashSet(rawOutputs), numberOfInvocations); + } + + /** + * Instantiates a new {@link SpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} method of + * the given {@link IRichSpout spout} in an infinite loop. The output type can be any type if parameter + * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is + * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared + * number of attributes. + * + * @param spout + * The {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. (Can be {@code null}.) + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public SpoutWrapper(final IRichSpout spout, final Collection rawOutputs) + throws IllegalArgumentException { + this(spout, rawOutputs, null); + } + + /** + * Instantiates a new {@link SpoutWrapper} that calls the {@link IRichSpout#nextTuple() nextTuple()} method of + * the given {@link IRichSpout spout} a finite number of times. The output type can be any type if parameter + * {@code rawOutput} is {@code true} and the spout's number of declared output tuples is 1. If {@code rawOutput} is + * {@code false} the output type will be one of {@link Tuple0} to {@link Tuple25} depending on the spout's declared + * number of attributes. + * + * @param spout + * The {@link IRichSpout spout} to be used. + * @param rawOutputs + * Contains stream names if a single attribute output stream, should not be of type {@link Tuple1} but be + * of a raw type. (Can be {@code null}.) + * @param numberOfInvocations + * The number of calls to {@link IRichSpout#nextTuple()}. If value is negative, {@link SpoutWrapper} + * terminates if no tuple was emitted for the first time. If value is {@code null}, finite invocation is + * disabled. + * @throws IllegalArgumentException + * If {@code rawOuput} is {@code true} and the number of declared output attributes is not 1 or if + * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range + * [0;25]. + */ + public SpoutWrapper(final IRichSpout spout, final Collection rawOutputs, + final Integer numberOfInvocations) throws IllegalArgumentException { + this.spout = spout; + this.numberOfAttributes = WrapperSetupHelper.getNumberOfAttributes(spout, rawOutputs); + this.numberOfInvocations = numberOfInvocations; + } + + /** + * Sets the original Storm topology. + * + * @param stormTopology + * The original Storm topology. + */ + public void setStormTopology(StormTopology stormTopology) { + this.stormTopology = stormTopology; + } + + @Override + public final void run(final SourceContext ctx) throws Exception { + this.collector = new SpoutCollector(this.numberOfAttributes, ctx); + + GlobalJobParameters config = super.getRuntimeContext().getExecutionConfig() + .getGlobalJobParameters(); + StormConfig stormConfig = new StormConfig(); + + if (config != null) { + if (config instanceof StormConfig) { + stormConfig = (StormConfig) config; + } else { + stormConfig.putAll(config.toMap()); + } + } + + this.spout.open(stormConfig, WrapperSetupHelper.createTopologyContext( + (StreamingRuntimeContext) super.getRuntimeContext(), this.spout, + this.stormTopology, stormConfig), new SpoutOutputCollector(this.collector)); + this.spout.activate(); + + if (numberOfInvocations == null) { + if (this.spout instanceof FiniteSpout) { + final FiniteSpout finiteSpout = (FiniteSpout) this.spout; + + while (this.isRunning && !finiteSpout.reachedEnd()) { + finiteSpout.nextTuple(); + } + } else { + while (this.isRunning) { + this.spout.nextTuple(); + } + } + } else { + int counter = this.numberOfInvocations; + if (counter >= 0) { + while ((--counter >= 0) && this.isRunning) { + this.spout.nextTuple(); + } + } else { + do { + this.collector.tupleEmitted = false; + this.spout.nextTuple(); + } while (this.collector.tupleEmitted && this.isRunning); + } + } + } + + /** + * {@inheritDoc} + *

    + * Sets the {@link #isRunning} flag to {@code false}. + */ + @Override + public void cancel() { + this.isRunning = false; + } + + @Override + public void close() throws Exception { + this.spout.close(); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormTuple.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/StormTuple.java similarity index 98% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormTuple.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/StormTuple.java index 07d94b4f8ce1d..c9ab8e584a2dc 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormTuple.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/StormTuple.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; /* * We do neither import @@ -53,7 +53,7 @@ class StormTuple implements backtype.storm.tuple.Tuple { * @param schema * The schema (ie, ordered field names) of the tuple. */ - public StormTuple(final IN flinkTuple, final Fields schema) { + StormTuple(final IN flinkTuple, final Fields schema) { if (flinkTuple instanceof org.apache.flink.api.java.tuple.Tuple) { this.schema = schema; final org.apache.flink.api.java.tuple.Tuple t = (org.apache.flink.api.java.tuple.Tuple) flinkTuple; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java similarity index 94% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java rename to flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java index 891497e874caf..d529b6a1900b7 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import backtype.storm.Config; import backtype.storm.generated.Bolt; @@ -29,8 +29,6 @@ import backtype.storm.topology.IRichSpout; import backtype.storm.tuple.Fields; -import org.apache.flink.stormcompatibility.util.FlinkOutputFieldsDeclarer; -import org.apache.flink.stormcompatibility.util.FlinkTopologyContext; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; import clojure.lang.Atom; @@ -43,16 +41,16 @@ import java.util.Map.Entry; /** - * {@link StormWrapperSetupHelper} is an helper class used by {@link AbstractStormSpoutWrapper} and - * {@link StormBoltWrapper}. + * {@link WrapperSetupHelper} is an helper class used by {@link AbstractStormSpoutWrapper} and + * {@link BoltWrapper}. */ -class StormWrapperSetupHelper { +class WrapperSetupHelper { /** The configuration key for the topology name. */ final static String TOPOLOGY_NAME = "storm.topology.name"; /** - * Computes the number of output attributes used by a {@link AbstractStormSpoutWrapper} or {@link StormBoltWrapper} + * Computes the number of output attributes used by a {@link AbstractStormSpoutWrapper} or {@link BoltWrapper} * per declared output stream. The number is {@code -1} for raw output type or a value within range [0;25] for * output type {@link org.apache.flink.api.java.tuple.Tuple0 Tuple0} to * {@link org.apache.flink.api.java.tuple.Tuple25 Tuple25}. @@ -68,7 +66,7 @@ class StormWrapperSetupHelper { * {@code rawOuput} is {@code false} and the number of declared output attributes is not with range * [0;25]. */ - public static HashMap getNumberOfAttributes(final IComponent spoutOrBolt, + static HashMap getNumberOfAttributes(final IComponent spoutOrBolt, final Collection rawOutputs) throws IllegalArgumentException { final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); @@ -112,7 +110,7 @@ public static HashMap getNumberOfAttributes(final IComponent sp * @return The created {@link TopologyContext}. */ @SuppressWarnings({ "rawtypes", "unchecked" }) - public static synchronized TopologyContext createTopologyContext( + static synchronized TopologyContext createTopologyContext( final StreamingRuntimeContext context, final IComponent spoutOrBolt, StormTopology stormTopology, Map stormConfig) { String operatorName = context.getTaskName(); @@ -161,7 +159,7 @@ public static synchronized TopologyContext createTopologyContext( } componentToSortedTasks.put(operatorName, sortedTasks); - FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); + SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); spoutOrBolt.declareOutputFields(declarer); componentToStreamToFields.put(operatorName, declarer.outputStreams); } else { diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java index 8e6356387429b..49de47601c2b6 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java @@ -14,14 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.api; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.stormcompatibility.util.FlinkOutputFieldsDeclarer; +import org.apache.flink.storm.api.FlinkOutputFieldsDeclarer; +import org.apache.flink.storm.util.AbstractTest; import org.junit.Assert; import org.junit.Test; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java similarity index 91% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java index 0dd9b1c293790..e6fb8e5ac77dc 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java @@ -14,11 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; -import org.apache.flink.stormcompatibility.util.TestDummyBolt; -import org.apache.flink.stormcompatibility.util.TestDummySpout; -import org.apache.flink.stormcompatibility.util.TestSink; +import org.apache.flink.storm.api.FlinkTopologyBuilder; +import org.apache.flink.storm.util.TestDummyBolt; +import org.apache.flink.storm.util.TestDummySpout; +import org.apache.flink.storm.util.TestSink; import org.junit.Test; import backtype.storm.tuple.Fields; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java index c98c9a3af0bac..9d04ca5695f4b 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; +import org.apache.flink.storm.api.FlinkTopology; import org.junit.Assert; import org.junit.Test; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java similarity index 96% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java index 2e4a53417704c..74ea67eade18b 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; import java.util.Map; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java index 146218fcc38ab..4abb60492a139 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; import java.util.Map; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestTopologyBuilder.java similarity index 91% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestTopologyBuilder.java index f664e584472ca..7bea94c0414b6 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestTopologyBuilder.java @@ -15,7 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.api; +package org.apache.flink.storm.api; + +import org.apache.flink.storm.api.FlinkTopologyBuilder; import backtype.storm.generated.StormTopology; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/AbstractTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/AbstractTest.java similarity index 96% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/AbstractTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/AbstractTest.java index 94a50cf1437a5..f51aba4bf216e 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/AbstractTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/AbstractTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import org.junit.Before; import org.slf4j.Logger; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java index 18918735beaef..1b320e5a642d1 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import backtype.storm.spout.SpoutOutputCollector; import backtype.storm.task.TopologyContext; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java similarity index 82% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java index c3cb7d7b7629f..17de427eac5ea 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java @@ -15,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import java.util.Iterator; -import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector; -import org.apache.flink.stormcompatibility.util.SplitStreamType; +import org.apache.flink.storm.util.SplitStreamType; +import org.apache.flink.storm.util.StormStreamSelector; import org.junit.Assert; import org.junit.Test; -public class FlinkStormStreamSelectorTest { +public class StormStreamSelectorTest { @Test public void testSelector() { - FlinkStormStreamSelector selector = new FlinkStormStreamSelector(); + StormStreamSelector selector = new StormStreamSelector(); SplitStreamType tuple = new SplitStreamType(); Iterator result; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java index b4993733a8320..b7458df75da5c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import java.util.Map; @@ -33,12 +33,15 @@ public class TestDummyBolt implements IRichBolt { public final static String groupingStreamId = "groupingStream"; private boolean emit = true; + @SuppressWarnings("rawtypes") + public Map config; private TopologyContext context; private OutputCollector collector; @SuppressWarnings("rawtypes") @Override public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + this.config = stormConf; this.context = context; this.collector = collector; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java index 345ca1224ec18..ed9ffffa5a433 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import java.util.Map; @@ -32,12 +32,15 @@ public class TestDummySpout implements IRichSpout { public final static String spoutStreamId = "spout-stream"; private boolean emit = true; + @SuppressWarnings("rawtypes") + public Map config; private TopologyContext context; private SpoutOutputCollector collector; @SuppressWarnings("rawtypes") @Override public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + this.config = conf; this.context = context; this.collector = collector; } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java index c8e5584b5b240..59939fd4b2b02 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.util; import java.util.LinkedList; import java.util.List; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java similarity index 69% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java index d01c3e0d21ecc..3d7d26b48fcef 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import backtype.storm.tuple.Values; import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.wrappers.BoltCollector; import org.apache.flink.streaming.api.operators.Output; import org.junit.Assert; import org.junit.Test; @@ -32,7 +33,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -public class StormBoltCollectorTest extends AbstractTest { +public class BoltCollectorTest extends AbstractTest { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test @@ -42,18 +43,18 @@ public void testBoltStormCollector() throws InstantiationException, IllegalAcces Tuple flinkTuple = null; final Values tuple = new Values(); - StormBoltCollector collector; + BoltCollector collector; final String streamId = "streamId"; HashMap attributes = new HashMap(); attributes.put(streamId, numberOfAttributes); if (numberOfAttributes == -1) { - collector = new StormBoltCollector(attributes, flinkCollector); + collector = new BoltCollector(attributes, flinkCollector); tuple.add(new Integer(this.r.nextInt())); } else { - collector = new StormBoltCollector(attributes, flinkCollector); + collector = new BoltCollector(attributes, flinkCollector); flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance(); for (int i = 0; i < numberOfAttributes; ++i) { @@ -76,30 +77,11 @@ public void testBoltStormCollector() throws InstantiationException, IllegalAcces } } - - @SuppressWarnings("unchecked") - @Test(expected = UnsupportedOperationException.class) - public void testReportError() { - new StormBoltCollector(mock(HashMap.class), mock(Output.class)).reportError(null); - } - @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testEmitDirect() { - new StormBoltCollector(mock(HashMap.class), mock(Output.class)).emitDirect(0, null, + new BoltCollector(mock(HashMap.class), mock(Output.class)).emitDirect(0, null, null, null); } - @SuppressWarnings("unchecked") - @Test(expected = UnsupportedOperationException.class) - public void testAck() { - new StormBoltCollector(mock(HashMap.class), mock(Output.class)).ack(null); - } - - @SuppressWarnings("unchecked") - @Test(expected = UnsupportedOperationException.class) - public void testFail() { - new StormBoltCollector(mock(HashMap.class), mock(Output.class)).fail(null); - } - } diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java similarity index 85% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java index 68175937b0202..e33fdb9a2ae8c 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import backtype.storm.task.OutputCollector; import backtype.storm.task.TopologyContext; @@ -29,14 +29,19 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.Configuration; -import org.apache.flink.stormcompatibility.util.AbstractTest; -import org.apache.flink.stormcompatibility.util.SplitStreamType; -import org.apache.flink.stormcompatibility.util.StormConfig; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.util.SplitStreamType; +import org.apache.flink.storm.util.StormConfig; +import org.apache.flink.storm.util.TestDummyBolt; +import org.apache.flink.storm.wrappers.BoltWrapper; +import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer; +import org.apache.flink.storm.wrappers.StormTuple; +import org.apache.flink.storm.wrappers.WrapperSetupHelper; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; - +import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.api.mockito.PowerMockito; @@ -45,6 +50,7 @@ import java.util.HashSet; import java.util.Map; +import java.util.Map.Entry; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; @@ -56,8 +62,8 @@ import static org.mockito.Mockito.when; @RunWith(PowerMockRunner.class) -@PrepareForTest({StreamRecordSerializer.class, StormWrapperSetupHelper.class}) -public class StormBoltWrapperTest extends AbstractTest { +@PrepareForTest({StreamRecordSerializer.class, WrapperSetupHelper.class}) +public class BoltWrapperTest extends AbstractTest { @Test(expected = IllegalArgumentException.class) public void testWrapperRawType() throws Exception { @@ -65,7 +71,7 @@ public void testWrapperRawType() throws Exception { declarer.declare(new Fields("dummy1", "dummy2")); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - new StormBoltWrapper(mock(IRichBolt.class), + new BoltWrapper(mock(IRichBolt.class), new String[] { Utils.DEFAULT_STREAM_ID }); } @@ -79,7 +85,7 @@ public void testWrapperToManyAttributes1() throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - new StormBoltWrapper(mock(IRichBolt.class)); + new BoltWrapper(mock(IRichBolt.class)); } @Test(expected = IllegalArgumentException.class) @@ -92,7 +98,7 @@ public void testWrapperToManyAttributes2() throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - new StormBoltWrapper(mock(IRichBolt.class), new String[] {}); + new BoltWrapper(mock(IRichBolt.class), new String[] {}); } @Test @@ -142,7 +148,7 @@ private void testWrapper(final int numberOfAttributes) throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, (Fields) null); + final BoltWrapper wrapper = new BoltWrapper(bolt, (Fields) null); wrapper.setup(mock(Output.class), taskContext); wrapper.open(null); @@ -179,7 +185,7 @@ public void testMultipleOutputStreams() throws Exception { raw.add("stream2"); } - final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, (Fields) null, raw); + final BoltWrapper wrapper = new BoltWrapper(bolt, (Fields) null, raw); wrapper.setup(output, taskContext); wrapper.open(null); @@ -224,9 +230,9 @@ public void testOpen() throws Exception { declarer.declare(new Fields("dummy")); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - final IRichBolt bolt = mock(IRichBolt.class); - final StormBoltWrapper wrapper = new StormBoltWrapper(bolt); + + BoltWrapper wrapper = new BoltWrapper(bolt); wrapper.setup(mock(Output.class), taskContext); // test without configuration @@ -239,9 +245,14 @@ public void testOpen() throws Exception { any(OutputCollector.class)); // test with Configuration + final TestDummyBolt testBolt = new TestDummyBolt(); + wrapper = new BoltWrapper(testBolt); + wrapper.setup(mock(Output.class), taskContext); + wrapper.open(null); - verify(bolt, times(3)).prepare(eq(flinkConfig.toMap()), any(TopologyContext.class), - any(OutputCollector.class)); + for (Entry entry : flinkConfig.toMap().entrySet()) { + Assert.assertEquals(entry.getValue(), testBolt.config.get(entry.getKey())); + } } @SuppressWarnings("unchecked") @@ -252,7 +263,7 @@ public void testOpenSink() throws Exception { final ExecutionConfig taskConfig = mock(ExecutionConfig.class); when(taskConfig.getGlobalJobParameters()).thenReturn(null).thenReturn(stormConfig) - .thenReturn(flinkConfig); + .thenReturn(flinkConfig); final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); when(taskContext.getExecutionConfig()).thenReturn(taskConfig); @@ -261,7 +272,7 @@ public void testOpenSink() throws Exception { final IRichBolt bolt = mock(IRichBolt.class); - final StormBoltWrapper wrapper = new StormBoltWrapper(bolt); + BoltWrapper wrapper = new BoltWrapper(bolt); wrapper.setup(mock(Output.class), taskContext); // test without configuration @@ -275,9 +286,14 @@ public void testOpenSink() throws Exception { isNull(OutputCollector.class)); // test with Configuration + final TestDummyBolt testBolt = new TestDummyBolt(); + wrapper = new BoltWrapper(testBolt); + wrapper.setup(mock(Output.class), taskContext); + wrapper.open(null); - verify(bolt, times(3)).prepare(eq(flinkConfig.toMap()), any(TopologyContext.class), - isNull(OutputCollector.class)); + for (Entry entry : flinkConfig.toMap().entrySet()) { + Assert.assertEquals(entry.getValue(), testBolt.config.get(entry.getKey())); + } } @SuppressWarnings("unchecked") @@ -289,7 +305,7 @@ public void testClose() throws Exception { declarer.declare(new Fields("dummy")); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - final StormBoltWrapper wrapper = new StormBoltWrapper(bolt); + final BoltWrapper wrapper = new BoltWrapper(bolt); final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); wrapper.setup(mock(Output.class), taskContext); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java similarity index 97% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java index bd9ea3f307c98..69d4a8e1f1824 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.util; +package org.apache.flink.storm.wrappers; import java.util.HashMap; @@ -27,7 +27,8 @@ import backtype.storm.metric.api.IMetric; import backtype.storm.metric.api.IReducer; -import org.apache.flink.stormcompatibility.util.FlinkTopologyContext; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.wrappers.FlinkTopologyContext; import org.junit.Test; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java similarity index 95% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java index 738eb1e9fa63b..4618101cf94bd 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; -import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer; import org.junit.Assert; import org.junit.Test; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java similarity index 78% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java index 36ed58aeb6365..6b60d2b02a592 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import backtype.storm.tuple.Values; import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.wrappers.SpoutCollector; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; import org.junit.Assert; import org.junit.Test; @@ -31,7 +32,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -public class StormSpoutCollectorTest extends AbstractTest { +public class SpoutCollectorTest extends AbstractTest { @SuppressWarnings({ "rawtypes", "unchecked" }) @Test @@ -41,18 +42,18 @@ public void testSpoutStormCollector() throws InstantiationException, IllegalAcce Tuple flinkTuple = null; final Values tuple = new Values(); - StormSpoutCollector collector; + SpoutCollector collector; final String streamId = "streamId"; HashMap attributes = new HashMap(); attributes.put(streamId, numberOfAttributes); if (numberOfAttributes == -1) { - collector = new StormSpoutCollector(attributes, flinkCollector); + collector = new SpoutCollector(attributes, flinkCollector); tuple.add(new Integer(this.r.nextInt())); } else { - collector = new StormSpoutCollector(attributes, flinkCollector); + collector = new SpoutCollector(attributes, flinkCollector); flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance(); for (int i = 0; i < numberOfAttributes; ++i) { @@ -76,17 +77,10 @@ public void testSpoutStormCollector() throws InstantiationException, IllegalAcce } } - @SuppressWarnings("unchecked") - @Test(expected = UnsupportedOperationException.class) - public void testReportError() { - new StormSpoutCollector(mock(HashMap.class), mock(SourceContext.class)) - .reportError(null); - } - @SuppressWarnings("unchecked") @Test(expected = UnsupportedOperationException.class) public void testEmitDirect() { - new StormSpoutCollector(mock(HashMap.class), mock(SourceContext.class)).emitDirect( + new SpoutCollector(mock(HashMap.class), mock(SourceContext.class)).emitDirect( 0, null, null, (Object) null); } diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java new file mode 100644 index 0000000000000..227d7363dde3f --- /dev/null +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java @@ -0,0 +1,220 @@ +/* + * 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.storm.wrappers; + +import backtype.storm.spout.SpoutOutputCollector; +import backtype.storm.task.TopologyContext; +import backtype.storm.topology.IRichSpout; +import backtype.storm.tuple.Fields; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.util.FiniteSpout; +import org.apache.flink.storm.util.FiniteTestSpout; +import org.apache.flink.storm.util.StormConfig; +import org.apache.flink.storm.util.TestDummySpout; +import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer; +import org.apache.flink.storm.wrappers.SpoutWrapper; +import org.apache.flink.storm.wrappers.WrapperSetupHelper; +import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; +import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.LinkedList; +import java.util.Map; +import java.util.Map.Entry; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(WrapperSetupHelper.class) +public class SpoutWrapperTest extends AbstractTest { + + @SuppressWarnings({ "rawtypes", "unchecked" }) + @Test + public void testRunPrepare() throws Exception { + final StormConfig stormConfig = new StormConfig(); + stormConfig.put(this.r.nextInt(), this.r.nextInt()); + final Configuration flinkConfig = new Configuration(); + flinkConfig.setInteger("testKey", this.r.nextInt()); + + final ExecutionConfig taskConfig = mock(ExecutionConfig.class); + when(taskConfig.getGlobalJobParameters()).thenReturn(null).thenReturn(stormConfig) + .thenReturn(flinkConfig); + + final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); + when(taskContext.getExecutionConfig()).thenReturn(taskConfig); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); + + final IRichSpout spout = mock(IRichSpout.class); + SpoutWrapper spoutWrapper = new SpoutWrapper(spout); + spoutWrapper.setRuntimeContext(taskContext); + spoutWrapper.cancel(); + + // test without configuration + spoutWrapper.run(mock(SourceContext.class)); + verify(spout).open(any(Map.class), any(TopologyContext.class), + any(SpoutOutputCollector.class)); + + // test with StormConfig + spoutWrapper.run(mock(SourceContext.class)); + verify(spout).open(eq(stormConfig), any(TopologyContext.class), + any(SpoutOutputCollector.class)); + + // test with Configuration + final TestDummySpout testSpout = new TestDummySpout(); + spoutWrapper = new SpoutWrapper(testSpout); + spoutWrapper.setRuntimeContext(taskContext); + spoutWrapper.cancel(); + + spoutWrapper.run(mock(SourceContext.class)); + for (Entry entry : flinkConfig.toMap().entrySet()) { + Assert.assertEquals(entry.getValue(), testSpout.config.get(entry.getKey())); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testRunExecuteFixedNumber() throws Exception { + final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer(); + declarer.declare(new Fields("dummy")); + PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments() + .thenReturn(declarer); + + final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); + when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); + + final IRichSpout spout = mock(IRichSpout.class); + final int numberOfCalls = this.r.nextInt(50); + final SpoutWrapper spoutWrapper = new SpoutWrapper(spout, + numberOfCalls); + spoutWrapper.setRuntimeContext(taskContext); + + spoutWrapper.run(mock(SourceContext.class)); + verify(spout, times(numberOfCalls)).nextTuple(); + } + + @Test + public void testRunExecuteFinite() throws Exception { + final int numberOfCalls = this.r.nextInt(50); + + final LinkedList> expectedResult = new LinkedList>(); + for (int i = numberOfCalls - 1; i >= 0; --i) { + expectedResult.add(new Tuple1(new Integer(i))); + } + + final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); + when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); + + final FiniteTestSpout spout = new FiniteTestSpout(numberOfCalls); + final SpoutWrapper> spoutWrapper = new SpoutWrapper>( + spout, -1); + spoutWrapper.setRuntimeContext(taskContext); + + final TestContext collector = new TestContext(); + spoutWrapper.run(collector); + + Assert.assertEquals(expectedResult, collector.result); + } + + @SuppressWarnings("unchecked") + @Test + public void runAndExecuteFiniteSpout() throws Exception { + final FiniteSpout stormSpout = mock(FiniteSpout.class); + when(stormSpout.reachedEnd()).thenReturn(false, false, false, true, false, false, true); + + final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); + when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); + + final SpoutWrapper wrapper = new SpoutWrapper(stormSpout); + wrapper.setRuntimeContext(taskContext); + + wrapper.run(mock(SourceContext.class)); + verify(stormSpout, times(3)).nextTuple(); + } + + @SuppressWarnings("unchecked") + @Test + public void runAndExecuteFiniteSpout2() throws Exception { + final FiniteSpout stormSpout = mock(FiniteSpout.class); + when(stormSpout.reachedEnd()).thenReturn(true, false, true, false, true, false, true); + + final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); + when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); + + final SpoutWrapper wrapper = new SpoutWrapper(stormSpout); + wrapper.setRuntimeContext(taskContext); + + wrapper.run(mock(SourceContext.class)); + verify(stormSpout, never()).nextTuple(); + } + + @Test + public void testCancel() throws Exception { + final int numberOfCalls = 5 + this.r.nextInt(5); + + final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class); + when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class)); + when(taskContext.getTaskStubParameters()).thenReturn(new Configuration()); + when(taskContext.getTaskName()).thenReturn("name"); + + final IRichSpout spout = new FiniteTestSpout(numberOfCalls); + + final SpoutWrapper> spoutWrapper = new SpoutWrapper>(spout); + spoutWrapper.setRuntimeContext(taskContext); + + spoutWrapper.cancel(); + final TestContext collector = new TestContext(); + spoutWrapper.run(collector); + + Assert.assertEquals(new LinkedList>(), collector.result); + } + + @Test + public void testClose() throws Exception { + final IRichSpout spout = mock(IRichSpout.class); + final SpoutWrapper> spoutWrapper = new SpoutWrapper>(spout); + + spoutWrapper.close(); + + verify(spout).close(); + } + +} diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java similarity index 99% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java index 06d539978875c..155fcd905db5a 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.stormcompatibility.util.AbstractTest; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.wrappers.StormTuple; import org.junit.Assert; import org.junit.Before; import org.junit.Test; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/TestContext.java similarity index 96% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/TestContext.java index 7c91e6f4be162..4c4749a979d72 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/TestContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java similarity index 92% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java rename to flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java index c799d63282037..19d05a80abbd9 100644 --- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.stormcompatibility.wrappers; +package org.apache.flink.storm.wrappers; import java.util.HashMap; import java.util.HashSet; @@ -34,11 +34,13 @@ import backtype.storm.tuple.Fields; import backtype.storm.utils.Utils; -import org.apache.flink.stormcompatibility.api.TestTopologyBuilder; -import org.apache.flink.stormcompatibility.util.AbstractTest; -import org.apache.flink.stormcompatibility.util.TestDummyBolt; -import org.apache.flink.stormcompatibility.util.TestDummySpout; -import org.apache.flink.stormcompatibility.util.TestSink; +import org.apache.flink.storm.api.TestTopologyBuilder; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.util.TestDummyBolt; +import org.apache.flink.storm.util.TestDummySpout; +import org.apache.flink.storm.util.TestSink; +import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer; +import org.apache.flink.storm.wrappers.WrapperSetupHelper; import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext; import org.junit.Assert; import org.junit.Test; @@ -55,8 +57,8 @@ @PowerMockIgnore("javax.*") @RunWith(PowerMockRunner.class) -@PrepareForTest(StormWrapperSetupHelper.class) -public class StormWrapperSetupHelperTest extends AbstractTest { +@PrepareForTest(WrapperSetupHelper.class) +public class WrapperSetupHelperTest extends AbstractTest { @Test public void testEmptyDeclarerBolt() { @@ -69,7 +71,7 @@ public void testEmptyDeclarerBolt() { } Assert.assertEquals(new HashMap(), - StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null)); + WrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null)); } @Test(expected = IllegalArgumentException.class) @@ -86,7 +88,7 @@ public void testRawType() throws Exception { declarer.declare(new Fields("dummy1", "dummy2")); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, + WrapperSetupHelper.getNumberOfAttributes(boltOrSpout, Sets.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID })); } @@ -108,7 +110,7 @@ public void testToManyAttributes() throws Exception { declarer.declare(new Fields(schema)); PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer); - StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null); + WrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null); } @Test @@ -143,7 +145,7 @@ private void testTupleTypes(final int numberOfAttributes) throws Exception { HashMap attributes = new HashMap(); attributes.put(Utils.DEFAULT_STREAM_ID, numberOfAttributes); - Assert.assertEquals(attributes, StormWrapperSetupHelper.getNumberOfAttributes( + Assert.assertEquals(attributes, WrapperSetupHelper.getNumberOfAttributes( boltOrSpout, numberOfAttributes == -1 ? Sets .newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }) : null)); @@ -228,9 +230,9 @@ public void testCreateTopologyContext() { taskCounter.put(thisComponentId, ++index); Config stormConfig = new Config(); - stormConfig.put(StormWrapperSetupHelper.TOPOLOGY_NAME, "test"); + stormConfig.put(WrapperSetupHelper.TOPOLOGY_NAME, "test"); - TopologyContext topologyContext = StormWrapperSetupHelper.createTopologyContext( + TopologyContext topologyContext = WrapperSetupHelper.createTopologyContext( context, operators.get(thisComponentId), stormTopology, stormConfig); ComponentCommon expcetedCommon = expectedContext.getComponentCommon(thisComponentId); diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j-test.properties b/flink-contrib/flink-storm/src/test/resources/log4j-test.properties similarity index 100% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j-test.properties rename to flink-contrib/flink-storm/src/test/resources/log4j-test.properties diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j.properties b/flink-contrib/flink-storm/src/test/resources/log4j.properties similarity index 100% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j.properties rename to flink-contrib/flink-storm/src/test/resources/log4j.properties diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/logback-test.xml b/flink-contrib/flink-storm/src/test/resources/logback-test.xml similarity index 100% rename from flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/logback-test.xml rename to flink-contrib/flink-storm/src/test/resources/logback-test.xml diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml index 9038363bae788..074edfa66bd4b 100644 --- a/flink-contrib/pom.xml +++ b/flink-contrib/pom.xml @@ -32,7 +32,8 @@ under the License. - flink-storm-compatibility + flink-storm + flink-storm-examples flink-streaming-contrib flink-tweet-inputformat flink-operator-stats