You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mj...@apache.org on 2015/10/06 13:31:26 UTC

[01/15] flink git commit: [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" *

Repository: flink
Updated Branches:
  refs/heads/master 39115abe4 -> 4cb967089


http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
new file mode 100644
index 0000000..19d05a8
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
@@ -0,0 +1,317 @@
+/*
+ * 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.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.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;
+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(WrapperSetupHelper.class)
+public class WrapperSetupHelperTest extends AbstractTest {
+
+	@Test
+	public void testEmptyDeclarerBolt() {
+		IComponent boltOrSpout;
+
+		if (this.r.nextBoolean()) {
+			boltOrSpout = mock(IRichSpout.class);
+		} else {
+			boltOrSpout = mock(IRichBolt.class);
+		}
+
+		Assert.assertEquals(new HashMap<String, Integer>(),
+				WrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null));
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testRawType() throws Exception {
+		IComponent boltOrSpout;
+
+		if (this.r.nextBoolean()) {
+			boltOrSpout = mock(IRichSpout.class);
+		} else {
+			boltOrSpout = mock(IRichBolt.class);
+		}
+
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+		declarer.declare(new Fields("dummy1", "dummy2"));
+		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		WrapperSetupHelper.getNumberOfAttributes(boltOrSpout,
+				Sets.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }));
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testToManyAttributes() throws Exception {
+		IComponent boltOrSpout;
+
+		if (this.r.nextBoolean()) {
+			boltOrSpout = mock(IRichSpout.class);
+		} else {
+			boltOrSpout = mock(IRichBolt.class);
+		}
+
+		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(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		WrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null);
+	}
+
+	@Test
+	public void testTupleTypes() throws Exception {
+		for (int i = -1; i < 26; ++i) {
+			this.testTupleTypes(i);
+		}
+	}
+
+	private void testTupleTypes(final int numberOfAttributes) throws Exception {
+		String[] schema;
+		if (numberOfAttributes == -1) {
+			schema = new String[1];
+		} else {
+			schema = new String[numberOfAttributes];
+		}
+		for (int i = 0; i < schema.length; ++i) {
+			schema[i] = "a" + i;
+		}
+
+		IComponent boltOrSpout;
+		if (this.r.nextBoolean()) {
+			boltOrSpout = mock(IRichSpout.class);
+		} else {
+			boltOrSpout = mock(IRichBolt.class);
+		}
+
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+		declarer.declare(new Fields(schema));
+		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		HashMap<String, Integer> attributes = new HashMap<String, Integer>();
+		attributes.put(Utils.DEFAULT_STREAM_ID, numberOfAttributes);
+
+		Assert.assertEquals(attributes, WrapperSetupHelper.getNumberOfAttributes(
+				boltOrSpout,
+				numberOfAttributes == -1 ? Sets
+						.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }) : null));
+	}
+
+	@Test
+	public void testCreateTopologyContext() {
+		HashMap<String, Integer> dops = new HashMap<String, Integer>();
+		dops.put("spout1", 1);
+		dops.put("spout2", 3);
+		dops.put("bolt1", 1);
+		dops.put("bolt2", 2);
+		dops.put("sink", 1);
+
+		HashMap<String, Integer> taskCounter = new HashMap<String, Integer>();
+		taskCounter.put("spout1", 0);
+		taskCounter.put("spout2", 0);
+		taskCounter.put("bolt1", 0);
+		taskCounter.put("bolt2", 0);
+		taskCounter.put("sink", 0);
+
+		HashMap<String, IComponent> operators = new HashMap<String, IComponent>();
+		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<Integer> taskIds = new HashSet<Integer>();
+
+		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(WrapperSetupHelper.TOPOLOGY_NAME, "test");
+
+			TopologyContext topologyContext = WrapperSetupHelper.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<Integer, String> taskToComponents = new HashMap<Integer, String>();
+			Set<Integer> allTaskIds = new HashSet<Integer>();
+			for (String componentId : expectedContext.getComponentIds()) {
+				List<Integer> possibleTasks = expectedContext.getComponentTasks(componentId);
+				List<Integer> tasks = topologyContext.getComponentTasks(componentId);
+
+				Iterator<Integer> p_it = possibleTasks.iterator();
+				Iterator<Integer> 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 */ }
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/resources/log4j-test.properties b/flink-contrib/flink-storm/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..0b686e5
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/resources/log4j-test.properties
@@ -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.
+################################################################################
+
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=OFF, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/resources/log4j.properties b/flink-contrib/flink-storm/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ed2bbcb
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/resources/log4j.properties
@@ -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.
+################################################################################
+
+# This file ensures that tests executed from the IDE show log output
+
+log4j.rootLogger=OFF, console
+
+# Log all infos in the given file
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target = System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/resources/logback-test.xml b/flink-contrib/flink-storm/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..4f56748
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/resources/logback-test.xml
@@ -0,0 +1,30 @@
+<!--
+  ~ 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.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml
index 9038363..074edfa 100644
--- a/flink-contrib/pom.xml
+++ b/flink-contrib/pom.xml
@@ -32,7 +32,8 @@ under the License.
 	</parent>
 
 	<modules>
-		<module>flink-storm-compatibility</module>
+		<module>flink-storm</module>
+		<module>flink-storm-examples</module>
 		<module>flink-streaming-contrib</module>
 		<module>flink-tweet-inputformat</module>
 		<module>flink-operator-stats</module>


[13/15] flink git commit: [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" *

Posted by mj...@apache.org.
[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


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4cb96708
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4cb96708
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4cb96708

Branch: refs/heads/master
Commit: 4cb9670892e34bd2321a2848f5195588e4442d0d
Parents: 7a67a60
Author: mjsax <mj...@apache.org>
Authored: Fri Oct 2 18:43:38 2015 +0200
Committer: mjsax <mj...@informatik.hu-berlin.de>
Committed: Tue Oct 6 13:29:32 2015 +0200

----------------------------------------------------------------------
 docs/apis/storm_compatibility.md                | 111 ++--
 .../flink-storm-compatibility-core/README.md    |  12 -
 .../flink-storm-compatibility-core/pom.xml      | 114 ----
 .../stormcompatibility/api/FlinkClient.java     | 315 ---------
 .../api/FlinkLocalCluster.java                  | 174 -----
 .../stormcompatibility/api/FlinkSubmitter.java  | 194 ------
 .../stormcompatibility/api/FlinkTopology.java   |  89 ---
 .../api/FlinkTopologyBuilder.java               | 407 ------------
 .../api/SplitStreamTypeKeySelector.java         |  47 --
 .../util/FiniteStormSpout.java                  |  39 --
 .../util/FlinkOutputFieldsDeclarer.java         | 168 -----
 .../util/FlinkStormStreamSelector.java          |  48 --
 .../util/FlinkTopologyContext.java              | 164 -----
 .../util/SplitStreamMapper.java                 |  39 --
 .../util/SplitStreamType.java                   |  52 --
 .../util/SplitStreamTypeKeySelector.java        |  46 --
 .../stormcompatibility/util/StormConfig.java    | 123 ----
 .../wrappers/AbstractStormCollector.java        | 143 ----
 .../wrappers/AbstractStormSpoutWrapper.java     | 153 -----
 .../wrappers/FiniteStormSpoutWrapper.java       | 111 ----
 .../wrappers/SetupOutputFieldsDeclarer.java     |  63 --
 .../wrappers/StormBoltCollector.java            |  94 ---
 .../wrappers/StormBoltWrapper.java              | 260 --------
 .../wrappers/StormFiniteSpoutWrapper.java       | 186 ------
 .../wrappers/StormSpoutCollector.java           |  82 ---
 .../wrappers/StormSpoutWrapper.java             | 105 ---
 .../stormcompatibility/wrappers/StormTuple.java | 327 ---------
 .../wrappers/StormWrapperSetupHelper.java       | 268 --------
 .../api/FlinkTopologyBuilderTest.java           |  76 ---
 .../api/FlinkTopologyTest.java                  |  62 --
 .../flink/stormcompatibility/api/TestBolt.java  |  48 --
 .../flink/stormcompatibility/api/TestSpout.java |  59 --
 .../api/TestTopologyBuilder.java                |  27 -
 .../stormcompatibility/util/AbstractTest.java   |  39 --
 .../util/FiniteTestSpout.java                   |  77 ---
 .../util/FlinkOutputFieldsDeclarerTest.java     | 193 ------
 .../util/FlinkStormStreamSelectorTest.java      |  51 --
 .../util/FlinkTopologyContextTest.java          | 114 ----
 .../stormcompatibility/util/TestDummyBolt.java  |  71 --
 .../stormcompatibility/util/TestDummySpout.java |  79 ---
 .../flink/stormcompatibility/util/TestSink.java |  60 --
 .../wrappers/FiniteStormSpoutWrapperTest.java   |  76 ---
 .../wrappers/SetupOutputFieldsDeclarerTest.java |  91 ---
 .../wrappers/StormBoltCollectorTest.java        | 105 ---
 .../wrappers/StormBoltWrapperTest.java          | 339 ----------
 .../wrappers/StormFiniteSpoutWrapperTest.java   | 117 ----
 .../wrappers/StormSpoutCollectorTest.java       |  94 ---
 .../wrappers/StormSpoutWrapperTest.java         | 120 ----
 .../wrappers/StormTupleTest.java                | 659 ------------------
 .../wrappers/StormWrapperSetupHelperTest.java   | 315 ---------
 .../wrappers/TestContext.java                   |  56 --
 .../src/test/resources/log4j-test.properties    |  27 -
 .../src/test/resources/log4j.properties         |  27 -
 .../src/test/resources/logback-test.xml         |  30 -
 .../README.md                                   |  20 -
 .../flink-storm-compatibility-examples/pom.xml  | 362 ----------
 .../excamation/ExclamationTopology.java         | 128 ----
 .../excamation/ExclamationWithStormBolt.java    | 143 ----
 .../excamation/ExclamationWithStormSpout.java   | 150 -----
 .../excamation/StormExclamationLocal.java       |  75 ---
 .../StormExclamationRemoteByClient.java         |  83 ---
 .../StormExclamationRemoteBySubmitter.java      |  81 ---
 .../stormoperators/ExclamationBolt.java         |  75 ---
 .../singlejoin/SingleJoinTopology.java          |  90 ---
 .../singlejoin/StormSingleJoinLocal.java        |  50 --
 .../singlejoin/stormoperators/AgeSpout.java     |  54 --
 .../singlejoin/stormoperators/GenderSpout.java  |  47 --
 .../stormoperators/SingleJoinBolt.java          | 132 ----
 .../split/SpoutSplitExample.java                | 102 ---
 .../split/stormoperators/RandomSpout.java       |  76 ---
 .../stormoperators/VerifyAndEnrichBolt.java     |  61 --
 .../util/AbstractStormBoltSink.java             |  76 ---
 .../util/AbstractStormSpout.java                |  70 --
 .../util/FiniteStormFileSpout.java              |  77 ---
 .../util/FiniteStormInMemorySpout.java          |  40 --
 .../util/OutputFormatter.java                   |  36 -
 .../util/SimpleOutputFormatter.java             |  42 --
 .../util/StormBoltFileSink.java                 |  76 ---
 .../util/StormBoltPrintSink.java                |  45 --
 .../stormcompatibility/util/StormFileSpout.java |  88 ---
 .../util/StormInMemorySpout.java                |  42 --
 .../util/TupleOutputFormatter.java              |  38 --
 .../wordcount/BoltTokenizerWordCount.java       | 122 ----
 .../wordcount/BoltTokenizerWordCountPojo.java   | 135 ----
 .../BoltTokenizerWordCountWithNames.java        | 138 ----
 .../wordcount/SpoutSourceWordCount.java         | 157 -----
 .../wordcount/StormWordCountLocal.java          |  75 ---
 .../wordcount/StormWordCountNamedLocal.java     |  76 ---
 .../wordcount/StormWordCountRemoteByClient.java |  85 ---
 .../StormWordCountRemoteBySubmitter.java        |  83 ---
 .../wordcount/WordCountTopology.java            | 135 ----
 .../stormoperators/StormBoltCounter.java        |  88 ---
 .../stormoperators/StormBoltCounterByName.java  |  88 ---
 .../stormoperators/StormBoltTokenizer.java      |  78 ---
 .../StormBoltTokenizerByName.java               |  78 ---
 .../stormoperators/StormWordCountFileSpout.java |  39 --
 .../StormWordCountInMemorySpout.java            |  40 --
 .../stormoperators/WordCountDataPojos.java      |  59 --
 .../stormoperators/WordCountDataTuple.java      |  34 -
 .../stormcompatibility/api/StormTestBase.java   | 117 ----
 .../ExclamationWithStormBoltITCase.java         |  49 --
 .../ExclamationWithStormSpoutITCase.java        |  46 --
 .../StormExclamationLocalITCase.java            |  48 --
 .../exclamation/util/ExclamationData.java       |  98 ---
 .../split/BoltSplitITCase.java                  |  28 -
 .../stormcompatibility/split/SplitBolt.java     |  61 --
 .../split/SplitBoltTopology.java                |  87 ---
 .../split/SplitSpoutTopology.java               |  85 ---
 .../split/SpoutSplitITCase.java                 |  28 -
 .../split/StormSplitStreamBoltLocal.java        |  51 --
 .../split/StormSplitStreamSpoutLocal.java       |  51 --
 .../wordcount/BoltTokenizerWordCountITCase.java |  45 --
 .../BoltTokenizerWordCountPojoITCase.java       |  45 --
 .../BoltTokenizerWordCountWithNamesITCase.java  |  45 --
 .../wordcount/SpoutSourceWordCountITCase.java   |  45 --
 .../wordcount/StormWordCountLocalITCase.java    |  45 --
 .../StormWordCountLocalNamedITCase.java         |  45 --
 .../src/test/resources/log4j-test.properties    |  27 -
 .../src/test/resources/log4j.properties         |  27 -
 .../src/test/resources/logback-test.xml         |  30 -
 flink-contrib/flink-storm-compatibility/pom.xml |  40 --
 flink-contrib/flink-storm-examples/README.md    |  20 +
 flink-contrib/flink-storm-examples/pom.xml      | 364 ++++++++++
 .../storm/excamation/ExclamationLocal.java      |  75 +++
 .../storm/excamation/ExclamationTopology.java   | 123 ++++
 .../storm/excamation/ExclamationWithBolt.java   | 144 ++++
 .../storm/excamation/ExclamationWithSpout.java  | 150 +++++
 .../excamation/operators/ExclamationBolt.java   |  75 +++
 .../flink/storm/split/SpoutSplitExample.java    | 102 +++
 .../storm/split/operators/RandomSpout.java      |  76 +++
 .../split/operators/VerifyAndEnrichBolt.java    |  61 ++
 .../flink/storm/util/AbstractBoltSink.java      |  76 +++
 .../flink/storm/util/AbstractLineSpout.java     |  70 ++
 .../apache/flink/storm/util/BoltFileSink.java   |  76 +++
 .../apache/flink/storm/util/BoltPrintSink.java  |  45 ++
 .../org/apache/flink/storm/util/FileSpout.java  |  88 +++
 .../flink/storm/util/FiniteFileSpout.java       |  77 +++
 .../flink/storm/util/FiniteInMemorySpout.java   |  40 ++
 .../apache/flink/storm/util/InMemorySpout.java  |  42 ++
 .../flink/storm/util/OutputFormatter.java       |  37 ++
 .../flink/storm/util/SimpleOutputFormatter.java |  41 ++
 .../flink/storm/util/TupleOutputFormatter.java  |  38 ++
 .../storm/wordcount/BoltTokenizerWordCount.java | 122 ++++
 .../wordcount/BoltTokenizerWordCountPojo.java   | 134 ++++
 .../BoltTokenizerWordCountWithNames.java        | 137 ++++
 .../storm/wordcount/SpoutSourceWordCount.java   | 157 +++++
 .../flink/storm/wordcount/WordCountLocal.java   |  76 +++
 .../storm/wordcount/WordCountLocalByName.java   |  77 +++
 .../wordcount/WordCountRemoteByClient.java      |  86 +++
 .../wordcount/WordCountRemoteBySubmitter.java   |  84 +++
 .../storm/wordcount/WordCountTopology.java      | 136 ++++
 .../storm/wordcount/operators/BoltCounter.java  |  90 +++
 .../wordcount/operators/BoltCounterByName.java  |  90 +++
 .../wordcount/operators/BoltTokenizer.java      |  78 +++
 .../operators/BoltTokenizerByName.java          |  78 +++
 .../wordcount/operators/WordCountDataPojos.java |  59 ++
 .../wordcount/operators/WordCountDataTuple.java |  34 +
 .../wordcount/operators/WordCountFileSpout.java |  39 ++
 .../operators/WordCountInMemorySpout.java       |  40 ++
 .../exclamation/ExclamationWithBoltITCase.java  |  49 ++
 .../exclamation/ExclamationWithSpoutITCase.java |  46 ++
 .../StormExclamationLocalITCase.java            |  48 ++
 .../storm/exclamation/util/ExclamationData.java |  98 +++
 .../flink/storm/split/BoltSplitITCase.java      |  28 +
 .../org/apache/flink/storm/split/SplitBolt.java |  61 ++
 .../flink/storm/split/SplitBoltTopology.java    |  87 +++
 .../flink/storm/split/SplitSpoutTopology.java   |  85 +++
 .../flink/storm/split/SplitStreamBoltLocal.java |  51 ++
 .../storm/split/SplitStreamSpoutLocal.java      |  51 ++
 .../flink/storm/split/SpoutSplitITCase.java     |  28 +
 .../apache/flink/storm/util/StormTestBase.java  | 117 ++++
 .../wordcount/BoltTokenizerWordCountITCase.java |  46 ++
 .../BoltTokenizerWordCountPojoITCase.java       |  46 ++
 .../BoltTokenizerWordCountWithNamesITCase.java  |  46 ++
 .../wordcount/SpoutSourceWordCountITCase.java   |  46 ++
 .../storm/wordcount/WordCountLocalITCase.java   |  46 ++
 .../wordcount/WordCountLocalNamedITCase.java    |  46 ++
 .../src/test/resources/log4j-test.properties    |  27 +
 .../src/test/resources/log4j.properties         |  27 +
 .../src/test/resources/logback-test.xml         |  30 +
 flink-contrib/flink-storm/README.md             |  15 +
 flink-contrib/flink-storm/pom.xml               | 114 ++++
 .../org/apache/flink/storm/api/FlinkClient.java | 315 +++++++++
 .../flink/storm/api/FlinkLocalCluster.java      | 173 +++++
 .../storm/api/FlinkOutputFieldsDeclarer.java    | 168 +++++
 .../apache/flink/storm/api/FlinkSubmitter.java  | 194 ++++++
 .../apache/flink/storm/api/FlinkTopology.java   |  89 +++
 .../flink/storm/api/FlinkTopologyBuilder.java   | 397 +++++++++++
 .../apache/flink/storm/util/FiniteSpout.java    |  36 +
 .../flink/storm/util/SplitStreamMapper.java     |  39 ++
 .../flink/storm/util/SplitStreamType.java       |  52 ++
 .../storm/util/SplitStreamTypeKeySelector.java  |  46 ++
 .../apache/flink/storm/util/StormConfig.java    | 123 ++++
 .../flink/storm/util/StormStreamSelector.java   |  48 ++
 .../storm/wrappers/AbstractStormCollector.java  | 143 ++++
 .../flink/storm/wrappers/BoltCollector.java     |  89 +++
 .../flink/storm/wrappers/BoltWrapper.java       | 260 ++++++++
 .../storm/wrappers/FlinkTopologyContext.java    | 165 +++++
 .../wrappers/SetupOutputFieldsDeclarer.java     |  66 ++
 .../flink/storm/wrappers/SpoutCollector.java    |  81 +++
 .../flink/storm/wrappers/SpoutWrapper.java      | 274 ++++++++
 .../apache/flink/storm/wrappers/StormTuple.java | 327 +++++++++
 .../storm/wrappers/WrapperSetupHelper.java      | 266 ++++++++
 .../api/FlinkOutputFieldsDeclarerTest.java      | 194 ++++++
 .../storm/api/FlinkTopologyBuilderTest.java     |  77 +++
 .../flink/storm/api/FlinkTopologyTest.java      |  63 ++
 .../org/apache/flink/storm/api/TestBolt.java    |  48 ++
 .../org/apache/flink/storm/api/TestSpout.java   |  59 ++
 .../flink/storm/api/TestTopologyBuilder.java    |  29 +
 .../apache/flink/storm/util/AbstractTest.java   |  39 ++
 .../flink/storm/util/FiniteTestSpout.java       |  77 +++
 .../storm/util/StormStreamSelectorTest.java     |  51 ++
 .../apache/flink/storm/util/TestDummyBolt.java  |  74 +++
 .../apache/flink/storm/util/TestDummySpout.java |  82 +++
 .../org/apache/flink/storm/util/TestSink.java   |  60 ++
 .../flink/storm/wrappers/BoltCollectorTest.java |  87 +++
 .../flink/storm/wrappers/BoltWrapperTest.java   | 355 ++++++++++
 .../wrappers/FlinkTopologyContextTest.java      | 115 ++++
 .../wrappers/SetupOutputFieldsDeclarerTest.java |  92 +++
 .../storm/wrappers/SpoutCollectorTest.java      |  88 +++
 .../flink/storm/wrappers/SpoutWrapperTest.java  | 220 +++++++
 .../flink/storm/wrappers/StormTupleTest.java    | 660 +++++++++++++++++++
 .../flink/storm/wrappers/TestContext.java       |  56 ++
 .../storm/wrappers/WrapperSetupHelperTest.java  | 317 +++++++++
 .../src/test/resources/log4j-test.properties    |  27 +
 .../src/test/resources/log4j.properties         |  27 +
 .../src/test/resources/logback-test.xml         |  30 +
 flink-contrib/pom.xml                           |   3 +-
 228 files changed, 10902 insertions(+), 11912 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/docs/apis/storm_compatibility.md
----------------------------------------------------------------------
diff --git a/docs/apis/storm_compatibility.md b/docs/apis/storm_compatibility.md
index d676db8..bf80d4e 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
 <dependency>
 	<groupId>org.apache.flink</groupId>
-	<artifactId>flink-storm-compatibility-core</artifactId>
+	<artifactId>flink-storm</artifactId>
 	<version>{{site.version}}</version>
 </dependency>
 ~~~
 
-**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<String>`).
@@ -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<OUT>` that serves as first argument to `addSource(...)`.
+The Spout object is handed to the constructor of `SpoutWrapper<OUT>` that serves as first argument to `addSource(...)`.
 The generic type declaration `OUT` specifies the type of the source output stream.
 
 <div class="codetabs" markdown="1">
@@ -114,7 +116,7 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironm
 
 // stream has `raw` type (single field output streams only)
 DataStream<String> rawInput = env.addSource(
-	new StormSpoutWrapper<String>(new StormFileSpout(localFilePath), new String[] { Utils.DEFAULT_STREAM_ID }), // emit default output stream as raw type
+	new SpoutWrapper<String>(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<String> rawInput = env.addSource(
 </div>
 </div>
 
-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<IN,OUT>` that serves as last argument to `transform(...)`.
+The Bolt object is handed to the constructor of `BoltWrapper<IN,OUT>` 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.
 
 <div class="codetabs" markdown="1">
@@ -143,7 +145,7 @@ DataStream<String> text = env.readTextFile(localFilePath);
 DataStream<Tuple2<String, Integer>> counts = text.transform(
 	"tokenizer", // operator name
 	TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)), // output type
-	new StormBoltWrapper<String, Tuple2<String, Integer>>(new StormBoltTokenizer())); // Bolt operator
+	new BoltWrapper<String, Tuple2<String, Integer>>(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<Tuple1<String>, Tuple2<String, Integer>>(new StormBoltTokenizerByName(), new Fields("sentence"))`.
+For this case, the constructor of `BoltWrapper` takes an additional argument: `new BoltWrapper<Tuple1<String>, ...>(..., new Fields("sentence"))`.
 The input type is `Tuple1<String>` 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 &ndash; it works as in regular Storm.
+If a whole topology is executed in Flink using `FlinkTopologyBuilder` etc., there is no special attention required &ndash; 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 &ndash; it works as in regular Storm.
+If a whole topology is executed in Flink using `FlinkTopologyBuilder` etc., there is no special attention required &ndash; it works as in regular Storm.
 
-For embedded usage, the output stream will be of data type `SplitStreamType<T>` and must be split by using `DataStream.split(...)` and `SplitDataStream.select(...)`.
-Flink provides the predefined output selector `FlinkStormStreamSelector<T>` for `.split(...)` already.
+For embedded usage, the output stream will be of data type `SplitStreamType<T>` and must be split by using `DataStream.split(...)` and `SplitStream.select(...)`.
+Flink provides the predefined output selector `StormStreamSelector<T>` for `.split(...)` already.
 Furthermore, the wrapper type `SplitStreamTuple<T>` can be removed using `SplitStreamMapper<T>`.
-If a data stream of type `SplitStreamTuple<T>` is used as input for a Bolt, `SplitStreamTuple<T>` must **not** be removed &ndash; `StormBoltWrapper` removes it automatically.
+If a data stream of type `SplitStreamTuple<T>` is used as input for a Bolt, it is **not** required to strip the wrapper &ndash; `BoltWrapper` removes it automatically.
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
@@ -217,11 +219,11 @@ If a data stream of type `SplitStreamTuple<T>` 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<SplitStreamType<SomeType>> multiStream = ...
 
-SplitDataStream<SplitStreamType<SomeType>> splitStream = multiStream.split(new FlinkStormStreamSelector<SomeType>());
+SplitStream<SplitStreamType<SomeType>> splitStream = multiStream.split(new StormStreamSelector<SomeType>());
 
 // remove SplitStreamMapper to get data stream of type SomeType
 DataStream<SomeType> s1 = splitStream.select("s1").map(new SplitStreamMapper<SomeType>).returns(SomeType.classs);
-// apply Bolt directly, without stripping SplitStreamMapper
+// apply Bolt directly, without stripping SplitStreamType
 DataStream<BoltOutputType> s2 = splitStream.select("s2").transform(/* use Bolt for further processing */);
 
 // do further processing on s1 and s2
@@ -230,67 +232,48 @@ DataStream<BoltOutputType> s2 = splitStream.select("s2").transform(/* use Bolt f
 </div>
 </div>
 
-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 &ndash; i.e., emit a finite number of records and stop after emitting the last record &ndash;, 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:
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
 ~~~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;
 	}
-	[...]
 }
 ~~~
 </div>
 </div>
 
-Using a `FiniteStormSpout` in a Flink topology:
-
-<div class="codetabs" markdown="1">
-<div data-lang="java" markdown="1">
-~~~java
-StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-DataStream<String> rawInput = env.addSource(
-	new FiniteStormSpoutWrapper<String>(new TimedFiniteStormSpout(), true)
-	TypeExtractor.getForClass(String.class));
-
-// process data stream
-[...]
-~~~
-</div>
-</div>
-
 # 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.

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md
----------------------------------------------------------------------
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 9663fc7..0000000
--- 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

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/pom.xml b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/pom.xml
deleted file mode 100644
index cced678..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/pom.xml
+++ /dev/null
@@ -1,114 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-storm-compatibility-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-storm-compatibility-core</artifactId>
-	<name>flink-storm-compatibility-core</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.storm</groupId>
-			<artifactId>storm-core</artifactId>
-			<version>0.9.4</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>log4j-over-slf4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<artifactId>logback-classic</artifactId>
-					<groupId>ch.qos.logback</groupId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-
-		<pluginManagement>
-			<plugins>
-				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-				<plugin>
-					<groupId>org.eclipse.m2e</groupId>
-					<artifactId>lifecycle-mapping</artifactId>
-					<version>1.0.0</version>
-					<configuration>
-						<lifecycleMappingMetadata>
-							<pluginExecutions>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.maven.plugins</groupId>
-										<artifactId>maven-dependency-plugin</artifactId>
-										<versionRange>[2.9,)</versionRange>
-										<goals>
-											<goal>unpack</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-							</pluginExecutions>
-						</lifecycleMappingMetadata>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java
deleted file mode 100644
index 4676102..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkClient.java
+++ /dev/null
@@ -1,315 +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 akka.actor.ActorRef;
-import akka.actor.ActorSystem;
-import akka.pattern.Patterns;
-import akka.util.Timeout;
-import backtype.storm.Config;
-import backtype.storm.generated.AlreadyAliveException;
-import backtype.storm.generated.InvalidTopologyException;
-import backtype.storm.generated.KillOptions;
-import backtype.storm.generated.Nimbus;
-import backtype.storm.generated.NotAliveException;
-import backtype.storm.utils.NimbusClient;
-import backtype.storm.utils.Utils;
-
-import com.google.common.collect.Lists;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.client.program.Client;
-import org.apache.flink.client.program.JobWithJars;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.client.JobStatusMessage;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobmanager.JobManager;
-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 scala.Some;
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.List;
-import java.util.Map;
-
-/**
- * {@link FlinkClient} mimics a Storm {@link NimbusClient} and {@link Nimbus}{@code .Client} at once, to interact with
- * Flink's JobManager instead of Storm's Nimbus.
- */
-public class FlinkClient {
-
-	/** The client's configuration */
-	private final Map<?,?> conf;
-	/** The jobmanager's host name */
-	private final String jobManagerHost;
-	/** The jobmanager's rpc port */
-	private final int jobManagerPort;
-	/** The user specified timeout in milliseconds */
-	private final String timeout;
-
-	// The following methods are derived from "backtype.storm.utils.NimbusClient"
-
-	/**
-	 * Instantiates a new {@link FlinkClient} for the given configuration, host name, and port. If values for {@link
-	 * Config#NIMBUS_HOST} and {@link Config#NIMBUS_THRIFT_PORT} of the given configuration are ignored.
-	 *
-	 * @param conf
-	 * 		A configuration.
-	 * @param host
-	 * 		The jobmanager's host name.
-	 * @param port
-	 * 		The jobmanager's rpc port.
-	 */
-	@SuppressWarnings("rawtypes")
-	public FlinkClient(final Map conf, final String host, final int port) {
-		this(conf, host, port, null);
-	}
-
-	/**
-	 * Instantiates a new {@link FlinkClient} for the given configuration, host name, and port. If values for {@link
-	 * Config#NIMBUS_HOST} and {@link Config#NIMBUS_THRIFT_PORT} of the given configuration are ignored.
-	 *
-	 * @param conf
-	 * 		A configuration.
-	 * @param host
-	 * 		The jobmanager's host name.
-	 * @param port
-	 * 		The jobmanager's rpc port.
-	 * @param timeout
-	 * 		Timeout
-	 */
-	@SuppressWarnings("rawtypes")
-	public FlinkClient(final Map conf, final String host, final int port, final Integer timeout) {
-		this.conf = conf;
-		this.jobManagerHost = host;
-		this.jobManagerPort = port;
-		if (timeout != null) {
-			this.timeout = timeout + " ms";
-		} else {
-			this.timeout = null;
-		}
-	}
-
-	/**
-	 * Returns a {@link FlinkClient} that uses the configured {@link Config#NIMBUS_HOST} and {@link
-	 * Config#NIMBUS_THRIFT_PORT} as JobManager address.
-	 *
-	 * @param conf
-	 * 		Configuration that contains the jobmanager's hostname and port.
-	 * @return A configured {@link FlinkClient}.
-	 */
-	@SuppressWarnings("rawtypes")
-	public static FlinkClient getConfiguredClient(final Map conf) {
-		final String nimbusHost = (String) conf.get(Config.NIMBUS_HOST);
-		final int nimbusPort = Utils.getInt(conf.get(Config.NIMBUS_THRIFT_PORT)).intValue();
-		return new FlinkClient(conf, nimbusHost, nimbusPort);
-	}
-
-	/**
-	 * Return a reference to itself.
-	 * <p/>
-	 * {@link FlinkClient} mimics both, {@link NimbusClient} and {@link Nimbus}{@code .Client}, at once.
-	 *
-	 * @return A reference to itself.
-	 */
-	public FlinkClient getClient() {
-		return this;
-	}
-
-	// The following methods are derived from "backtype.storm.generated.Nimubs.Client"
-
-	/**
-	 * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink does not support
-	 * uploading a jar file before hand. Jar files are always uploaded directly when a program is submitted.
-	 */
-	public void submitTopology(final String name, final String uploadedJarLocation, final FlinkTopology topology)
-			throws AlreadyAliveException, InvalidTopologyException {
-		this.submitTopologyWithOpts(name, uploadedJarLocation, topology);
-	}
-
-	/**
-	 * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink does not support
-	 * uploading a jar file before hand. Jar files are always uploaded directly when a program is submitted.
-	 */
-	public void submitTopologyWithOpts(final String name, final String uploadedJarLocation, final FlinkTopology
-			topology)
-					throws AlreadyAliveException, InvalidTopologyException {
-
-		if (this.getTopologyJobId(name) != null) {
-			throw new AlreadyAliveException();
-		}
-
-		final File uploadedJarFile = new File(uploadedJarLocation);
-		try {
-			JobWithJars.checkJarFile(uploadedJarFile);
-		} catch (final IOException e) {
-			throw new RuntimeException("Problem with jar file " + uploadedJarFile.getAbsolutePath(), e);
-		}
-
-		/* set storm configuration */
-		if (this.conf != null) {
-			topology.getConfig().setGlobalJobParameters(new StormConfig(this.conf));
-		}
-
-		final JobGraph jobGraph = topology.getStreamGraph().getJobGraph(name);
-		jobGraph.addJar(new Path(uploadedJarFile.getAbsolutePath()));
-
-		final Configuration configuration = jobGraph.getJobConfiguration();
-		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost);
-		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
-
-		final Client client;
-		try {
-			client = new Client(configuration);
-		} catch (IOException e) {
-			throw new RuntimeException("Could not establish a connection to the job manager", e);
-		}
-
-		try {
-			ClassLoader classLoader = JobWithJars.buildUserCodeClassLoader(
-					Lists.newArrayList(uploadedJarFile),
-					this.getClass().getClassLoader());
-			client.runDetached(jobGraph, classLoader);
-		} catch (final ProgramInvocationException e) {
-			throw new RuntimeException("Cannot execute job due to ProgramInvocationException", e);
-		}
-	}
-
-	public void killTopology(final String name) throws NotAliveException {
-		this.killTopologyWithOpts(name, null);
-	}
-
-	public void killTopologyWithOpts(final String name, final KillOptions options) throws NotAliveException {
-		final JobID jobId = this.getTopologyJobId(name);
-		if (jobId == null) {
-			throw new NotAliveException();
-		}
-
-		try {
-			final ActorRef jobManager = this.getJobManager();
-
-			if (options != null) {
-				try {
-					Thread.sleep(1000 * options.get_wait_secs());
-				} catch (final InterruptedException e) {
-					throw new RuntimeException(e);
-				}
-			}
-
-			final FiniteDuration askTimeout = this.getTimeout();
-			final Future<Object> response = Patterns.ask(jobManager, new CancelJob(jobId), new Timeout(askTimeout));
-			try {
-				Await.result(response, askTimeout);
-			} catch (final Exception e) {
-				throw new RuntimeException("Killing topology " + name + " with Flink job ID " + jobId + " failed", e);
-			}
-		} catch (final IOException e) {
-			throw new RuntimeException("Could not connect to Flink JobManager with address " + this.jobManagerHost
-					+ ":" + this.jobManagerPort, e);
-		}
-	}
-
-	// Flink specific additional methods
-
-	/**
-	 * Package internal method to get a Flink {@link JobID} from a Storm topology name.
-	 *
-	 * @param id
-	 * 		The Storm topology name.
-	 * @return Flink's internally used {@link JobID}.
-	 */
-	JobID getTopologyJobId(final String id) {
-		final Configuration configuration = GlobalConfiguration.getConfiguration();
-		if (this.timeout != null) {
-			configuration.setString(ConfigConstants.AKKA_ASK_TIMEOUT, this.timeout);
-		}
-
-		try {
-			final ActorRef jobManager = this.getJobManager();
-
-			final FiniteDuration askTimeout = this.getTimeout();
-			final Future<Object> response = Patterns.ask(jobManager, JobManagerMessages.getRequestRunningJobsStatus(),
-					new Timeout(askTimeout));
-
-			Object result;
-			try {
-				result = Await.result(response, askTimeout);
-			} catch (final Exception e) {
-				throw new RuntimeException("Could not retrieve running jobs from the JobManager", e);
-			}
-
-			if (result instanceof RunningJobsStatus) {
-				final List<JobStatusMessage> jobs = ((RunningJobsStatus) result).getStatusMessages();
-
-				for (final JobStatusMessage status : jobs) {
-					if (status.getJobName().equals(id)) {
-						return status.getJobId();
-					}
-				}
-			} else {
-				throw new RuntimeException("ReqeustRunningJobs requires a response of type "
-						+ "RunningJobs. Instead the response is of type " + result.getClass() + ".");
-			}
-		} catch (final IOException e) {
-			throw new RuntimeException("Could not connect to Flink JobManager with address " + this.jobManagerHost
-					+ ":" + this.jobManagerPort, e);
-		}
-
-		return null;
-	}
-
-	private FiniteDuration getTimeout() {
-		final Configuration configuration = GlobalConfiguration.getConfiguration();
-		if (this.timeout != null) {
-			configuration.setString(ConfigConstants.AKKA_ASK_TIMEOUT, this.timeout);
-		}
-
-		return AkkaUtils.getTimeout(configuration);
-	}
-
-	private ActorRef getJobManager() throws IOException {
-		final Configuration configuration = GlobalConfiguration.getConfiguration();
-
-		ActorSystem actorSystem;
-		try {
-			final scala.Tuple2<String, Object> systemEndpoint = new scala.Tuple2<String, Object>("", 0);
-			actorSystem = AkkaUtils.createActorSystem(configuration, new Some<scala.Tuple2<String, Object>>(
-					systemEndpoint));
-		} catch (final Exception e) {
-			throw new RuntimeException("Could not start actor system to communicate with JobManager", e);
-		}
-
-		return JobManager.getJobManagerActorRef(new InetSocketAddress(this.jobManagerHost, this.jobManagerPort),
-				actorSystem, AkkaUtils.getLookupTimeout(configuration));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
deleted file mode 100644
index 9b3fb54..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkLocalCluster.java
+++ /dev/null
@@ -1,174 +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.LocalCluster;
-import backtype.storm.generated.ClusterSummary;
-import backtype.storm.generated.KillOptions;
-import backtype.storm.generated.RebalanceOptions;
-import backtype.storm.generated.StormTopology;
-import backtype.storm.generated.SubmitOptions;
-import backtype.storm.generated.TopologyInfo;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.StreamingMode;
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-import java.util.Objects;
-
-/**
- * {@link FlinkLocalCluster} mimics a Storm {@link LocalCluster}.
- */
-public class FlinkLocalCluster {
-
-	/** The log used by this mini cluster */
-	private static final Logger LOG = LoggerFactory.getLogger(FlinkLocalCluster.class);
-
-	/** The flink mini cluster on which to execute the programs */
-	private final FlinkMiniCluster flink;
-
-
-	public FlinkLocalCluster() {
-		this.flink = new LocalFlinkMiniCluster(new Configuration(), true, StreamingMode.STREAMING);
-		this.flink.start();
-	}
-
-	public FlinkLocalCluster(FlinkMiniCluster flink) {
-		this.flink = Objects.requireNonNull(flink);
-	}
-
-	@SuppressWarnings("rawtypes")
-	public void submitTopology(final String topologyName, final Map conf, final FlinkTopology topology)
-			throws Exception {
-		this.submitTopologyWithOpts(topologyName, conf, topology, null);
-	}
-
-	@SuppressWarnings("rawtypes")
-	public void submitTopologyWithOpts(final String topologyName, final Map conf, final FlinkTopology topology, final SubmitOptions submitOpts) throws Exception {
-		LOG.info("Running Storm topology on FlinkLocalCluster");
-
-		if(conf != null) {
-			topology.getConfig().setGlobalJobParameters(new StormConfig(conf));
-		}
-
-		JobGraph jobGraph = topology.getStreamGraph().getJobGraph(topologyName);
-		this.flink.submitJobDetached(jobGraph);
-	}
-
-	public void killTopology(final String topologyName) {
-		this.killTopologyWithOpts(topologyName, null);
-	}
-
-	public void killTopologyWithOpts(final String name, final KillOptions options) {
-	}
-
-	public void activate(final String topologyName) {
-	}
-
-	public void deactivate(final String topologyName) {
-	}
-
-	public void rebalance(final String name, final RebalanceOptions options) {
-	}
-
-	public void shutdown() {
-		flink.stop();
-	}
-
-	public String getTopologyConf(final String id) {
-		return null;
-	}
-
-	public StormTopology getTopology(final String id) {
-		return null;
-	}
-
-	public ClusterSummary getClusterInfo() {
-		return null;
-	}
-
-	public TopologyInfo getTopologyInfo(final String id) {
-		return null;
-	}
-
-	public Map<?, ?> getState() {
-		return null;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Access to default local cluster
-	// ------------------------------------------------------------------------
-
-	// A different {@link FlinkLocalCluster} to be used for execution of ITCases
-	private static LocalClusterFactory currentFactory = new DefaultLocalClusterFactory();
-
-	/**
-	 * Returns a {@link FlinkLocalCluster} that should be used for execution. If no cluster was set by
-	 * {@link #initialize(LocalClusterFactory)} in advance, a new {@link FlinkLocalCluster} is returned.
-	 *
-	 * @return a {@link FlinkLocalCluster} to be used for execution
-	 */
-	public static FlinkLocalCluster getLocalCluster() {
-		return currentFactory.createLocalCluster();
-	}
-
-	/**
-	 * Sets a different factory for FlinkLocalClusters to be used for execution.
-	 *
-	 * @param clusterFactory
-	 * 		The LocalClusterFactory to create the local clusters for execution.
-	 */
-	public static void initialize(LocalClusterFactory clusterFactory) {
-		currentFactory = Objects.requireNonNull(clusterFactory);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Cluster factory
-	// ------------------------------------------------------------------------
-
-	/**
-	 * A factory that creates local clusters.
-	 */
-	public static interface LocalClusterFactory {
-
-		/**
-		 * Creates a local flink cluster.
-		 * @return A local flink cluster.
-		 */
-		FlinkLocalCluster createLocalCluster();
-	}
-
-	/**
-	 * A factory that instantiates a FlinkLocalCluster.
-	 */
-	public static class DefaultLocalClusterFactory implements LocalClusterFactory {
-
-		@Override
-		public FlinkLocalCluster createLocalCluster() {
-			return new FlinkLocalCluster();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java
deleted file mode 100644
index 5f3f31e..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkSubmitter.java
+++ /dev/null
@@ -1,194 +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.Config;
-import backtype.storm.StormSubmitter;
-import backtype.storm.generated.AlreadyAliveException;
-import backtype.storm.generated.InvalidTopologyException;
-import backtype.storm.generated.SubmitOptions;
-import backtype.storm.utils.Utils;
-
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.client.program.ContextEnvironment;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.json.simple.JSONValue;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.util.Map;
-
-/**
- * {@link FlinkSubmitter} mimics a {@link StormSubmitter} to submit Storm topologies to a Flink cluster.
- */
-public class FlinkSubmitter {
-	public final static Logger logger = LoggerFactory.getLogger(FlinkSubmitter.class);
-
-	/**
-	 * Submits a topology to run on the cluster. A topology runs forever or until explicitly killed.
-	 *
-	 * @param name
-	 * 		the name of the storm.
-	 * @param stormConf
-	 * 		the topology-specific configuration. See {@link Config}.
-	 * @param topology
-	 * 		the processing to execute.
-	 * @param opts
-	 * 		to manipulate the starting of the topology.
-	 * @throws AlreadyAliveException
-	 * 		if a topology with this name is already running
-	 * @throws InvalidTopologyException
-	 * 		if an invalid topology was submitted
-	 */
-	public static void submitTopology(final String name, final Map<?, ?> stormConf, final FlinkTopology topology,
-			final SubmitOptions opts)
-					throws AlreadyAliveException, InvalidTopologyException {
-		submitTopology(name, stormConf, topology);
-	}
-
-	/**
-	 * Submits a topology to run on the cluster. A topology runs forever or until explicitly killed. The given {@link
-	 * FlinkProgressListener} is ignored because progress bars are not supported by Flink.
-	 *
-	 * @param name
-	 * 		the name of the storm.
-	 * @param stormConf
-	 * 		the topology-specific configuration. See {@link Config}.
-	 * @param topology
-	 * 		the processing to execute.
-	 * @throws AlreadyAliveException
-	 * 		if a topology with this name is already running
-	 * @throws InvalidTopologyException
-	 * 		if an invalid topology was submitted
-	 */
-	@SuppressWarnings({"rawtypes", "unchecked"})
-	public static void submitTopology(final String name, final Map stormConf, final FlinkTopology topology)
-			throws AlreadyAliveException, InvalidTopologyException {
-		if (!Utils.isValidConf(stormConf)) {
-			throw new IllegalArgumentException("Storm conf is not valid. Must be json-serializable");
-		}
-
-		final Configuration flinkConfig = GlobalConfiguration.getConfiguration();
-		if (!stormConf.containsKey(Config.NIMBUS_HOST)) {
-			stormConf.put(Config.NIMBUS_HOST,
-					flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost"));
-		}
-		if (!stormConf.containsKey(Config.NIMBUS_THRIFT_PORT)) {
-			stormConf.put(Config.NIMBUS_THRIFT_PORT,
-					new Integer(flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-							6123)));
-		}
-
-		final String serConf = JSONValue.toJSONString(stormConf);
-
-		final FlinkClient client = FlinkClient.getConfiguredClient(stormConf);
-		try {
-			if (client.getTopologyJobId(name) != null) {
-				throw new RuntimeException("Topology with name `" + name + "` already exists on cluster");
-			}
-			String localJar = System.getProperty("storm.jar");
-			if (localJar == null) {
-				try {
-					for (final File file : ((ContextEnvironment) ExecutionEnvironment.getExecutionEnvironment())
-							.getJars()) {
-						// TODO verify that there is only one jar
-						localJar = file.getAbsolutePath();
-					}
-				} catch (final ClassCastException e) {
-					// ignore
-				}
-			}
-
-			logger.info("Submitting topology " + name + " in distributed mode with conf " + serConf);
-			client.submitTopologyWithOpts(name, localJar, topology);
-		} catch (final InvalidTopologyException e) {
-			logger.warn("Topology submission exception: " + e.get_msg());
-			throw e;
-		} catch (final AlreadyAliveException e) {
-			logger.warn("Topology already alive exception", e);
-			throw e;
-		}
-
-		logger.info("Finished submitting topology: " + name);
-	}
-
-	/**
-	 * Same as {@link #submitTopology(String, Map, FlinkTopology, SubmitOptions)}. Progress bars are not supported by
-	 * Flink.
-	 *
-	 * @param name
-	 * 		the name of the storm.
-	 * @param stormConf
-	 * 		the topology-specific configuration. See {@link Config}.
-	 * @param topology
-	 * 		the processing to execute.
-	 * @throws AlreadyAliveException
-	 * 		if a topology with this name is already running
-	 * @throws InvalidTopologyException
-	 * 		if an invalid topology was submitted
-	 */
-	public static void submitTopologyWithProgressBar(final String name, final Map<?, ?> stormConf,
-			final FlinkTopology topology)
-					throws AlreadyAliveException, InvalidTopologyException {
-		submitTopology(name, stormConf, topology);
-	}
-
-	/**
-	 * In Flink, jar files are submitted directly when a program is started. Thus, this method does nothing. The
-	 * returned value is parameter localJar, because this give the best integration of Storm behavior within a Flink
-	 * environment.
-	 *
-	 * @param conf
-	 * 		the topology-specific configuration. See {@link Config}.
-	 * @param localJar
-	 * 		file path of the jar file to submit
-	 * @return the value of parameter localJar
-	 */
-	@SuppressWarnings("rawtypes")
-	public static String submitJar(final Map conf, final String localJar) {
-		return submitJar(localJar);
-	}
-
-	/**
-	 * In Flink, jar files are submitted directly when a program is started. Thus, this method does nothing. The
-	 * returned value is parameter localJar, because this give the best integration of Storm behavior within a Flink
-	 * environment.
-	 *
-	 * @param localJar
-	 * 		file path of the jar file to submit
-	 * @return the value of parameter localJar
-	 */
-	public static String submitJar(final String localJar) {
-		if (localJar == null) {
-			throw new RuntimeException(
-					"Must submit topologies using the 'storm' client script so that StormSubmitter knows which jar " +
-					"to upload");
-		}
-
-		return localJar;
-	}
-
-	/**
-	 * Dummy interface use to track progress of file upload. Does not do anything. Kept for compatibility.
-	 */
-	public interface FlinkProgressListener {
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 179466e..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java
+++ /dev/null
@@ -1,89 +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.generated.StormTopology;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
-/**
- * {@link FlinkTopology} mimics a {@link StormTopology} and is implemented in terms of a {@link
- * StreamExecutionEnvironment} . In contrast to a regular {@link StreamExecutionEnvironment}, a {@link FlinkTopology}
- * cannot be executed directly, but must be handed over to a {@link FlinkLocalCluster}, {@link FlinkSubmitter}, or
- * {@link FlinkClient}.
- */
-public class FlinkTopology extends StreamExecutionEnvironment {
-
-	/** The number of declared tasks for the whole program (ie, sum over all dops) */
-	private int numberOfTasks = 0;
-
-	public FlinkTopology() {
-		// Set default parallelism to 1, to mirror Storm default behavior
-		super.setParallelism(1);
-	}
-
-	/**
-	 * Is not supported. In order to execute use {@link FlinkLocalCluster}, {@link FlinkSubmitter}, or {@link
-	 * FlinkClient}.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public JobExecutionResult execute() throws Exception {
-		throw new UnsupportedOperationException(
-				"A FlinkTopology cannot be executed directly. Use FlinkLocalCluster, FlinkSubmitter, or FlinkClient " +
-				"instead.");
-	}
-
-	/**
-	 * Is not supported. In order to execute use {@link FlinkLocalCluster}, {@link FlinkSubmitter} or {@link
-	 * FlinkClient}.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public JobExecutionResult execute(final String jobName) throws Exception {
-		throw new UnsupportedOperationException(
-				"A FlinkTopology cannot be executed directly. Use FlinkLocalCluster, FlinkSubmitter, or FlinkClient " +
-				"instead.");
-	}
-
-	/**
-	 * Increased the number of declared tasks of this program by the given value.
-	 *
-	 * @param dop
-	 * 		The dop of a new operator that increases the number of overall tasks.
-	 */
-	public void increaseNumberOfTasks(final int dop) {
-		assert (dop > 0);
-		this.numberOfTasks += dop;
-	}
-
-	/**
-	 * Return the number or required tasks to execute this program.
-	 *
-	 * @return the number or required tasks to execute this program
-	 */
-	public int getNumberOfTasks() {
-		return this.numberOfTasks;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
----------------------------------------------------------------------
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
deleted file mode 100644
index d62d56b..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
+++ /dev/null
@@ -1,407 +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.generated.ComponentCommon;
-import backtype.storm.generated.GlobalStreamId;
-import backtype.storm.generated.Grouping;
-import backtype.storm.generated.StormTopology;
-import backtype.storm.topology.BasicBoltExecutor;
-import backtype.storm.topology.BoltDeclarer;
-import backtype.storm.topology.IBasicBolt;
-import backtype.storm.topology.IRichBolt;
-import backtype.storm.topology.IRichSpout;
-import backtype.storm.topology.IRichStateSpout;
-import backtype.storm.topology.SpoutDeclarer;
-import backtype.storm.topology.TopologyBuilder;
-import backtype.storm.tuple.Fields;
-
-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.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-
-/**
- * {@link FlinkTopologyBuilder} mimics a {@link TopologyBuilder}, but builds a Flink program instead of a Storm
- * topology. Most methods (except {@link #createTopology()} are copied from the original {@link TopologyBuilder}
- * implementation to ensure equal behavior.<br />
- * <br />
- * <strong>CAUTION: {@link IRichStateSpout StateSpout}s are currently not supported.</strong>
- */
-public class FlinkTopologyBuilder {
-
-	/** A Storm {@link TopologyBuilder} to build a real Storm topology */
-	private final TopologyBuilder stormBuilder = new TopologyBuilder();
-	/** All user spouts by their ID */
-	private final HashMap<String, IRichSpout> spouts = new HashMap<String, IRichSpout>();
-	/** All user bolts by their ID */
-	private final HashMap<String, IRichBolt> bolts = new HashMap<String, IRichBolt>();
-	/** All declared streams and output schemas by operator ID */
-	private final HashMap<String, HashMap<String, Fields>> outputStreams = new HashMap<String, HashMap<String, Fields>>();
-	/** All spouts&bolts declarers by their ID */
-	private final HashMap<String, FlinkOutputFieldsDeclarer> declarers = new HashMap<String, FlinkOutputFieldsDeclarer>();
-	// 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() {
-		this.stormTopology = this.stormBuilder.createTopology();
-
-		final FlinkTopology env = new FlinkTopology();
-		env.setParallelism(1);
-
-		final HashMap<String, HashMap<String, DataStream>> availableInputs = new HashMap<String, HashMap<String, DataStream>>();
-
-		for (final Entry<String, IRichSpout> spout : this.spouts.entrySet()) {
-			final String spoutId = spout.getKey();
-			final IRichSpout userSpout = spout.getValue();
-
-			final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-			userSpout.declareOutputFields(declarer);
-			final HashMap<String,Fields> sourceStreams = declarer.outputStreams;
-			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);
-			}
-			spoutWrapper.setStormTopology(stormTopology);
-
-			DataStreamSource source;
-			HashMap<String, DataStream> outputStreams = new HashMap<String, DataStream>();
-			if (sourceStreams.size() == 1) {
-				final String outputStreamId = (String) sourceStreams.keySet().toArray()[0];
-				source = env.addSource(spoutWrapper, spoutId,
-						declarer.getOutputType(outputStreamId));
-				outputStreams.put(outputStreamId, source);
-			} else {
-				source = env.addSource(spoutWrapper, spoutId,
-						TypeExtractor.getForClass(SplitStreamType.class));
-				SplitStream splitSource = source.split(new FlinkStormStreamSelector());
-
-				for (String streamId : sourceStreams.keySet()) {
-					outputStreams.put(streamId, splitSource.select(streamId));
-				}
-			}
-			availableInputs.put(spoutId, outputStreams);
-
-			int dop = 1;
-			final ComponentCommon common = stormTopology.get_spouts().get(spoutId).get_common();
-			if (common.is_set_parallelism_hint()) {
-				dop = common.get_parallelism_hint();
-				source.setParallelism(dop);
-			} else {
-				common.set_parallelism_hint(1);
-			}
-			env.increaseNumberOfTasks(dop);
-		}
-
-		final HashMap<String, IRichBolt> unprocessedBolts = new HashMap<String, IRichBolt>();
-		unprocessedBolts.putAll(this.bolts);
-
-		final HashMap<String, Set<Entry<GlobalStreamId, Grouping>>> unprocessdInputsPerBolt =
-				new HashMap<String, Set<Entry<GlobalStreamId, Grouping>>>();
-
-		/* Because we do not know the order in which an iterator steps over a set, we might process a consumer before
-		 * its producer
-		 * ->thus, we might need to repeat multiple times
-		 */
-		boolean makeProgress = true;
-		while (unprocessedBolts.size() > 0) {
-			if (!makeProgress) {
-				throw new RuntimeException(
-						"Unable to build Topology. Could not connect the following bolts: "
-								+ unprocessedBolts.keySet());
-			}
-			makeProgress = false;
-
-			final Iterator<Entry<String, IRichBolt>> boltsIterator = unprocessedBolts.entrySet().iterator();
-			while (boltsIterator.hasNext()) {
-
-				final Entry<String, IRichBolt> bolt = boltsIterator.next();
-				final String boltId = bolt.getKey();
-				final IRichBolt userBolt = bolt.getValue();
-
-				final ComponentCommon common = stormTopology.get_bolts().get(boltId).get_common();
-
-				Set<Entry<GlobalStreamId, Grouping>> unprocessedInputs = unprocessdInputsPerBolt.get(boltId);
-				if (unprocessedInputs == null) {
-					unprocessedInputs = new HashSet<Entry<GlobalStreamId, Grouping>>();
-					unprocessedInputs.addAll(common.get_inputs().entrySet());
-					unprocessdInputsPerBolt.put(boltId, unprocessedInputs);
-				}
-
-				// connect each available producer to the current bolt
-				final Iterator<Entry<GlobalStreamId, Grouping>> inputStreamsIterator = unprocessedInputs.iterator();
-				while (inputStreamsIterator.hasNext()) {
-
-					final Entry<GlobalStreamId, Grouping> stormInputStream = inputStreamsIterator.next();
-					final String producerId = stormInputStream.getKey().get_componentId();
-					final String inputStreamId = stormInputStream.getKey().get_streamId();
-
-					HashMap<String, DataStream> producer = availableInputs.get(producerId);
-					if (producer != null) {
-						makeProgress = true;
-
-						DataStream inputStream = producer.get(inputStreamId);
-						if (inputStream != null) {
-							final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-							userBolt.declareOutputFields(declarer);
-							final HashMap<String, Fields> boltOutputStreams = declarer.outputStreams;
-							this.outputStreams.put(boltId, boltOutputStreams);
-							this.declarers.put(boltId, declarer);
-
-							// if producer was processed already
-							final Grouping grouping = stormInputStream.getValue();
-							if (grouping.is_set_shuffle()) {
-								// Storm uses a round-robin shuffle strategy
-								inputStream = inputStream.rebalance();
-							} else if (grouping.is_set_fields()) {
-								// global grouping is emulated in Storm via an empty fields grouping list
-								final List<String> fields = grouping.get_fields();
-								if (fields.size() > 0) {
-									FlinkOutputFieldsDeclarer prodDeclarer = this.declarers.get(producerId);
-									if (producer.size() == 1) {
-										inputStream = inputStream.keyBy(prodDeclarer
-												.getGroupingFieldIndexes(inputStreamId,
-														grouping.get_fields()));
-									} else {
-										inputStream = inputStream
-												.keyBy(new SplitStreamTypeKeySelector(
-														prodDeclarer.getGroupingFieldIndexes(
-																inputStreamId,
-																grouping.get_fields())));
-									}
-								} else {
-									inputStream = inputStream.global();
-								}
-							} else if (grouping.is_set_all()) {
-								inputStream = inputStream.broadcast();
-							} else if (!grouping.is_set_local_or_shuffle()) {
-								throw new UnsupportedOperationException(
-										"Flink only supports (local-or-)shuffle, fields, all, and global grouping");
-							}
-
-							SingleOutputStreamOperator outputStream;
-							StormBoltWrapper boltWrapper;
-							if (boltOutputStreams.size() < 2) { // single output stream or sink
-								String outputStreamId = null;
-								if (boltOutputStreams.size() == 1) {
-									outputStreamId = (String) boltOutputStreams.keySet().toArray()[0];
-								}
-								final TypeInformation<?> outType = declarer
-										.getOutputType(outputStreamId);
-
-								boltWrapper = new StormBoltWrapper(userBolt, this.outputStreams
-										.get(producerId).get(inputStreamId));
-								outputStream = inputStream.transform(boltId, outType, boltWrapper);
-
-								if (outType != null) {
-									// only for non-sink nodes
-									HashMap<String, DataStream> op = new HashMap<String, DataStream>();
-									op.put(outputStreamId, outputStream);
-									availableInputs.put(boltId, op);
-								}
-							} else {
-								final TypeInformation<?> outType = TypeExtractor
-										.getForClass(SplitStreamType.class);
-
-								boltWrapper = new StormBoltWrapper(userBolt, this.outputStreams.get(producerId).get(inputStreamId));
-								outputStream = inputStream.transform(boltId, outType, boltWrapper);
-
-								SplitStream splitStreams = outputStream
-										.split(new FlinkStormStreamSelector());
-
-								HashMap<String, DataStream> op = new HashMap<String, DataStream>();
-								for (String outputStreamId : boltOutputStreams.keySet()) {
-									op.put(outputStreamId, splitStreams.select(outputStreamId));
-								}
-								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);
-
-							inputStreamsIterator.remove();
-						} else {
-							throw new RuntimeException("Cannot connect '" + boltId + "' to '"
-									+ producerId + "'. Stream '" + inputStreamId + "' not found.");
-						}
-					}
-				}
-
-				if (unprocessedInputs.size() == 0) {
-					// all inputs are connected; processing bolt completed
-					boltsIterator.remove();
-				}
-			}
-		}
-		return env;
-	}
-
-	/**
-	 * Define a new bolt in this topology with parallelism of just one thread.
-	 *
-	 * @param id
-	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
-	 * 		outputs.
-	 * @param bolt
-	 * 		the bolt
-	 * @return use the returned object to declare the inputs to this component
-	 */
-	public BoltDeclarer setBolt(final String id, final IRichBolt bolt) {
-		return this.setBolt(id, bolt, null);
-	}
-
-	/**
-	 * Define a new bolt in this topology with the specified amount of parallelism.
-	 *
-	 * @param id
-	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
-	 * 		outputs.
-	 * @param bolt
-	 * 		the bolt
-	 * @param parallelism_hint
-	 * 		the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a
-	 * 		process somewhere around the cluster.
-	 * @return use the returned object to declare the inputs to this component
-	 */
-	public BoltDeclarer setBolt(final String id, final IRichBolt bolt, final Number parallelism_hint) {
-		final BoltDeclarer declarer = this.stormBuilder.setBolt(id, bolt, parallelism_hint);
-		this.bolts.put(id, bolt);
-		return declarer;
-	}
-
-	/**
-	 * Define a new bolt in this topology. This defines a basic bolt, which is a simpler to use but more restricted
-	 * kind
-	 * of bolt. Basic bolts are intended for non-aggregation processing and automate the anchoring/acking process to
-	 * achieve proper reliability in the topology.
-	 *
-	 * @param id
-	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
-	 * 		outputs.
-	 * @param bolt
-	 * 		the basic bolt
-	 * @return use the returned object to declare the inputs to this component
-	 */
-	public BoltDeclarer setBolt(final String id, final IBasicBolt bolt) {
-		return this.setBolt(id, bolt, null);
-	}
-
-	/**
-	 * Define a new bolt in this topology. This defines a basic bolt, which is a simpler to use but more restricted
-	 * kind
-	 * of bolt. Basic bolts are intended for non-aggregation processing and automate the anchoring/acking process to
-	 * achieve proper reliability in the topology.
-	 *
-	 * @param id
-	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
-	 * 		outputs.
-	 * @param bolt
-	 * 		the basic bolt
-	 * @param parallelism_hint
-	 * 		the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a
-	 * 		process somwehere around the cluster.
-	 * @return use the returned object to declare the inputs to this component
-	 */
-	public BoltDeclarer setBolt(final String id, final IBasicBolt bolt, final Number parallelism_hint) {
-		return this.setBolt(id, new BasicBoltExecutor(bolt), parallelism_hint);
-	}
-
-	/**
-	 * Define a new spout in this topology.
-	 *
-	 * @param id
-	 * 		the id of this component. This id is referenced by other components that want to consume this spout's
-	 * 		outputs.
-	 * @param spout
-	 * 		the spout
-	 */
-	public SpoutDeclarer setSpout(final String id, final IRichSpout spout) {
-		return this.setSpout(id, spout, null);
-	}
-
-	/**
-	 * Define a new spout in this topology with the specified parallelism. If the spout declares itself as
-	 * non-distributed, the parallelism_hint will be ignored and only one task will be allocated to this component.
-	 *
-	 * @param id
-	 * 		the id of this component. This id is referenced by other components that want to consume this spout's
-	 * 		outputs.
-	 * @param parallelism_hint
-	 * 		the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a
-	 * 		process somwehere around the cluster.
-	 * @param spout
-	 * 		the spout
-	 */
-	public SpoutDeclarer setSpout(final String id, final IRichSpout spout, final Number parallelism_hint) {
-		final SpoutDeclarer declarer = this.stormBuilder.setSpout(id, spout, parallelism_hint);
-		this.spouts.put(id, spout);
-		return declarer;
-	}
-
-	// TODO add StateSpout support (Storm 0.9.4 does not yet support StateSpouts itself)
-	/* not implemented by Storm 0.9.4
-	 * public void setStateSpout(final String id, final IRichStateSpout stateSpout) {
-	 * this.stormBuilder.setStateSpout(id, stateSpout);
-	 * }
-	 * public void setStateSpout(final String id, final IRichStateSpout stateSpout, final Number parallelism_hint) {
-	 * this.stormBuilder.setStateSpout(id, stateSpout, parallelism_hint);
-	 * }
-	 */
-
-	// for internal testing purpose only
-	StormTopology getStormTopology() {
-		return this.stormTopology;
-	}
-}


[10/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 6817593..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java
+++ /dev/null
@@ -1,339 +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.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.Values;
-import backtype.storm.utils.Utils;
-
-import org.apache.flink.api.common.ExecutionConfig;
-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.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.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.HashSet;
-import java.util.Map;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.isNull;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.same;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({StreamRecordSerializer.class, StormWrapperSetupHelper.class})
-public class StormBoltWrapperTest extends AbstractTest {
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testWrapperRawType() throws Exception {
-		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-		declarer.declare(new Fields("dummy1", "dummy2"));
-		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		new StormBoltWrapper<Object, Object>(mock(IRichBolt.class),
-				new String[] { Utils.DEFAULT_STREAM_ID });
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testWrapperToManyAttributes1() throws Exception {
-		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(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		new StormBoltWrapper<Object, Object>(mock(IRichBolt.class));
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testWrapperToManyAttributes2() throws Exception {
-		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(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		new StormBoltWrapper<Object, Object>(mock(IRichBolt.class), new String[] {});
-	}
-
-	@Test
-	public void testWrapper() throws Exception {
-		for (int i = -1; i < 26; ++i) {
-			this.testWrapper(i);
-		}
-	}
-
-	@SuppressWarnings({"rawtypes", "unchecked"})
-	private void testWrapper(final int numberOfAttributes) throws Exception {
-		assert ((-1 <= numberOfAttributes) && (numberOfAttributes <= 25));
-		Tuple flinkTuple = null;
-		String rawTuple = null;
-
-		if (numberOfAttributes == -1) {
-			rawTuple = "test";
-		} else {
-			flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
-		}
-
-		final String[] schema;
-		if (numberOfAttributes == -1) {
-			schema = new String[1];
-		} else {
-			schema = new String[numberOfAttributes];
-		}
-		for (int i = 0; i < schema.length; ++i) {
-			schema[i] = "a" + i;
-		}
-
-		final StreamRecord record = mock(StreamRecord.class);
-		if (numberOfAttributes == -1) {
-			when(record.getValue()).thenReturn(rawTuple);
-		} else {
-			when(record.getValue()).thenReturn(flinkTuple);
-		}
-
-		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 SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-		declarer.declare(new Fields(schema));
-		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, (Fields) null);
-		wrapper.setup(mock(Output.class), taskContext);
-		wrapper.open(null);
-
-		wrapper.processElement(record);
-		if (numberOfAttributes == -1) {
-			verify(bolt).execute(eq(new StormTuple<String>(rawTuple, null)));
-		} else {
-			verify(bolt).execute(eq(new StormTuple<Tuple>(flinkTuple, null)));
-		}
-	}
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testMultipleOutputStreams() throws Exception {
-		final boolean rawOutType1 = super.r.nextBoolean();
-		final boolean rawOutType2 = super.r.nextBoolean();
-
-		final StreamRecord record = mock(StreamRecord.class);
-		when(record.getValue()).thenReturn(2).thenReturn(3);
-
-		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);
-
-		final TestBolt bolt = new TestBolt();
-		final HashSet<String> raw = new HashSet<String>();
-		if (rawOutType1) {
-			raw.add("stream1");
-		}
-		if (rawOutType2) {
-			raw.add("stream2");
-		}
-
-		final StormBoltWrapper wrapper = new StormBoltWrapper(bolt, (Fields) null, raw);
-		wrapper.setup(output, taskContext);
-		wrapper.open(null);
-
-		final SplitStreamType splitRecord = new SplitStreamType<Integer>();
-		if (rawOutType1) {
-			splitRecord.streamId = "stream1";
-			splitRecord.value = 2;
-		} else {
-			splitRecord.streamId = "stream1";
-			splitRecord.value = new Tuple1<Integer>(2);
-		}
-		wrapper.processElement(record);
-		verify(output).collect(new StreamRecord<SplitStreamType>(splitRecord, 0));
-
-		if (rawOutType2) {
-			splitRecord.streamId = "stream2";
-			splitRecord.value = 3;
-		} else {
-			splitRecord.streamId = "stream2";
-			splitRecord.value = new Tuple1<Integer>(3);
-		}
-		wrapper.processElement(record);
-		verify(output, times(2)).collect(new StreamRecord<SplitStreamType>(splitRecord, 0));
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testOpen() 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);
-		when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
-		when(taskContext.getTaskName()).thenReturn("name");
-
-		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-		declarer.declare(new Fields("dummy"));
-		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-
-		final IRichBolt bolt = mock(IRichBolt.class);
-		final StormBoltWrapper<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
-		wrapper.setup(mock(Output.class), taskContext);
-
-		// test without configuration
-		wrapper.open(null);
-		verify(bolt).prepare(any(Map.class), any(TopologyContext.class), any(OutputCollector.class));
-
-		// test with StormConfig
-		wrapper.open(null);
-		verify(bolt).prepare(same(stormConfig), any(TopologyContext.class),
-				any(OutputCollector.class));
-
-		// test with Configuration
-		wrapper.open(null);
-		verify(bolt, times(3)).prepare(eq(flinkConfig.toMap()), any(TopologyContext.class),
-				any(OutputCollector.class));
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testOpenSink() 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);
-		when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
-		when(taskContext.getTaskName()).thenReturn("name");
-
-		final IRichBolt bolt = mock(IRichBolt.class);
-
-		final StormBoltWrapper<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
-		wrapper.setup(mock(Output.class), taskContext);
-
-		// test without configuration
-		wrapper.open(null);
-		verify(bolt).prepare(any(Map.class), any(TopologyContext.class),
-				isNull(OutputCollector.class));
-
-		// test with StormConfig
-		wrapper.open(null);
-		verify(bolt).prepare(same(stormConfig), any(TopologyContext.class),
-				isNull(OutputCollector.class));
-
-		// test with Configuration
-		wrapper.open(null);
-		verify(bolt, times(3)).prepare(eq(flinkConfig.toMap()), any(TopologyContext.class),
-				isNull(OutputCollector.class));
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test
-	public void testClose() throws Exception {
-		final IRichBolt bolt = mock(IRichBolt.class);
-
-		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-		declarer.declare(new Fields("dummy"));
-		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		final StormBoltWrapper<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
-
-		final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
-		wrapper.setup(mock(Output.class), taskContext);
-
-		wrapper.close();
-		wrapper.dispose();
-
-		verify(bolt).cleanup();
-	}
-
-	private static final class TestBolt implements IRichBolt {
-		private static final long serialVersionUID = 7278692872260138758L;
-		private OutputCollector collector;
-
-		@SuppressWarnings("rawtypes")
-		@Override
-		public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-			this.collector = collector;
-		}
-
-		int counter = 0;
-		@Override
-		public void execute(backtype.storm.tuple.Tuple input) {
-			if (++counter % 2 == 1) {
-				this.collector.emit("stream1", new Values(input.getInteger(0)));
-			} else {
-				this.collector.emit("stream2", new Values(input.getInteger(0)));
-			}
-		}
-
-		@Override
-		public void cleanup() {}
-
-		@Override
-		public void declareOutputFields(OutputFieldsDeclarer declarer) {
-			declarer.declareStream("stream1", new Fields("a1"));
-			declarer.declareStream("stream2", new Fields("a2"));
-		}
-
-		@Override
-		public Map<String, Object> getComponentConfiguration() {
-			return null;
-		}
-	}
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java
----------------------------------------------------------------------
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 77f1b05..0000000
--- 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<Object>(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<Tuple1<Integer>> expectedResult = new LinkedList<Tuple1<Integer>>();
-		for (int i = numberOfCalls - 1; i >= 0; --i) {
-			expectedResult.add(new Tuple1<Integer>(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<Tuple1<Integer>> spoutWrapper = new StormFiniteSpoutWrapper<Tuple1<Integer>>(
-				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<Tuple1<Integer>> expectedResult = new LinkedList<Tuple1<Integer>>();
-		expectedResult.add(new Tuple1<Integer>(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<Tuple1<Integer>> spoutWrapper = new StormFiniteSpoutWrapper<Tuple1<Integer>>(
-				spout);
-
-		spoutWrapper.cancel();
-		final TestContext collector = new TestContext();
-		spoutWrapper.run(collector);
-
-		Assert.assertEquals(expectedResult, collector.result);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java
deleted file mode 100644
index 36ed58a..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollectorTest.java
+++ /dev/null
@@ -1,94 +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.tuple.Values;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.stormcompatibility.util.AbstractTest;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.HashMap;
-import java.util.List;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-
-public class StormSpoutCollectorTest extends AbstractTest {
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testSpoutStormCollector() throws InstantiationException, IllegalAccessException {
-		for (int numberOfAttributes = -1; numberOfAttributes < 26; ++numberOfAttributes) {
-			final SourceContext flinkCollector = mock(SourceContext.class);
-			Tuple flinkTuple = null;
-			final Values tuple = new Values();
-
-			StormSpoutCollector<?> collector;
-
-			final String streamId = "streamId";
-			HashMap<String, Integer> attributes = new HashMap<String, Integer>();
-			attributes.put(streamId, numberOfAttributes);
-
-			if (numberOfAttributes == -1) {
-				collector = new StormSpoutCollector(attributes, flinkCollector);
-				tuple.add(new Integer(this.r.nextInt()));
-
-			} else {
-				collector = new StormSpoutCollector(attributes, flinkCollector);
-				flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
-
-				for (int i = 0; i < numberOfAttributes; ++i) {
-					tuple.add(new Integer(this.r.nextInt()));
-					flinkTuple.setField(tuple.get(i), i);
-				}
-			}
-
-			final List<Integer> taskIds;
-			final Object messageId = new Integer(this.r.nextInt());
-
-			taskIds = collector.emit(streamId, tuple, messageId);
-
-			Assert.assertNull(taskIds);
-
-			if (numberOfAttributes == -1) {
-				verify(flinkCollector).collect(tuple.get(0));
-			} else {
-				verify(flinkCollector).collect(flinkTuple);
-			}
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test(expected = UnsupportedOperationException.class)
-	public void testReportError() {
-		new StormSpoutCollector<Object>(mock(HashMap.class), mock(SourceContext.class))
-				.reportError(null);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test(expected = UnsupportedOperationException.class)
-	public void testEmitDirect() {
-		new StormSpoutCollector<Object>(mock(HashMap.class), mock(SourceContext.class)).emitDirect(
-				0, null, null,
-				(Object) null);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java
----------------------------------------------------------------------
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 f4fb4da..0000000
--- 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<Tuple1<Integer>> spoutWrapper = new StormSpoutWrapper<Tuple1<Integer>>(spout);
-		spoutWrapper.setRuntimeContext(taskContext);
-
-		spoutWrapper.cancel();
-		final TestContext collector = new TestContext();
-		spoutWrapper.run(collector);
-
-		Assert.assertEquals(new LinkedList<Tuple1<Integer>>(), collector.result);
-	}
-
-	@Test
-	public void testClose() throws Exception {
-		final IRichSpout spout = mock(IRichSpout.class);
-		final StormSpoutWrapper<Tuple1<Integer>> spoutWrapper = new StormSpoutWrapper<Tuple1<Integer>>(spout);
-
-		spoutWrapper.close();
-
-		verify(spout).close();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java
deleted file mode 100644
index 06d5399..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormTupleTest.java
+++ /dev/null
@@ -1,659 +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.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.stormcompatibility.util.AbstractTest;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.mockito.Mockito.mock;
-
-public class StormTupleTest extends AbstractTest {
-	private static final String fieldName = "fieldName";
-	private static final String fieldNamePojo = "member";
-
-	private int arity, index;
-
-	@Override
-	@Before
-	public void prepare() {
-		super.prepare();
-		this.arity = 1 + r.nextInt(25);
-		this.index = r.nextInt(this.arity);
-	}
-
-	@Test
-	public void nonTupleTest() {
-		final Object flinkTuple = this.r.nextInt();
-
-		final StormTuple<Object> tuple = new StormTuple<Object>(flinkTuple, null);
-		Assert.assertSame(flinkTuple, tuple.getValue(0));
-
-		final List<Object> values = tuple.getValues();
-		Assert.assertEquals(1, values.size());
-		Assert.assertEquals(flinkTuple, values.get(0));
-	}
-
-	@Test
-	public void tupleTest() throws InstantiationException, IllegalAccessException {
-		final int numberOfAttributes = this.r.nextInt(26);
-		final Object[] data = new Object[numberOfAttributes];
-
-		final Tuple flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			data[i] = this.r.nextInt();
-			flinkTuple.setField(data[i], i);
-		}
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		final List<Object> values = tuple.getValues();
-
-		Assert.assertEquals(numberOfAttributes, values.size());
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			Assert.assertEquals(flinkTuple.getField(i), values.get(i));
-		}
-
-		Assert.assertEquals(numberOfAttributes, tuple.size());
-	}
-
-	@Test
-	public void testBinary() {
-		final byte[] data = new byte[this.r.nextInt(15)];
-		this.r.nextBytes(data);
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getBinary(index));
-	}
-
-	@Test
-	public void testBoolean() {
-		final Boolean flinkTuple = this.r.nextBoolean();
-
-		final StormTuple<Boolean> tuple = new StormTuple<Boolean>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getBoolean(0));
-	}
-
-	@Test
-	public void testByte() {
-		final Byte flinkTuple = (byte) this.r.nextInt();
-
-		final StormTuple<Byte> tuple = new StormTuple<Byte>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getByte(0));
-	}
-
-	@Test
-	public void testDouble() {
-		final Double flinkTuple = this.r.nextDouble();
-
-		final StormTuple<Double> tuple = new StormTuple<Double>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getDouble(0));
-	}
-
-	@Test
-	public void testFloat() {
-		final Float flinkTuple = this.r.nextFloat();
-
-		final StormTuple<Float> tuple = new StormTuple<Float>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getFloat(0));
-	}
-
-	@Test
-	public void testInteger() {
-		final Integer flinkTuple = this.r.nextInt();
-
-		final StormTuple<Integer> tuple = new StormTuple<Integer>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getInteger(0));
-	}
-
-	@Test
-	public void testLong() {
-		final Long flinkTuple = this.r.nextLong();
-
-		final StormTuple<Long> tuple = new StormTuple<Long>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getLong(0));
-	}
-
-	@Test
-	public void testShort() {
-		final Short flinkTuple = (short) this.r.nextInt();
-
-		final StormTuple<Short> tuple = new StormTuple<Short>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getShort(0));
-	}
-
-	@Test
-	public void testString() {
-		final byte[] data = new byte[this.r.nextInt(15)];
-		this.r.nextBytes(data);
-		final String flinkTuple = new String(data);
-
-		final StormTuple<String> tuple = new StormTuple<String>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple, tuple.getString(0));
-	}
-
-	@Test
-	public void testBinaryTuple() {
-		final byte[] data = new byte[this.r.nextInt(15)];
-		this.r.nextBytes(data);
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getBinary(index));
-	}
-
-	@Test
-	public void testBooleanTuple() {
-		final Boolean data = this.r.nextBoolean();
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getBoolean(index));
-	}
-
-	@Test
-	public void testByteTuple() {
-		final Byte data = (byte) this.r.nextInt();
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getByte(index));
-	}
-
-	@Test
-	public void testDoubleTuple() {
-		final Double data = this.r.nextDouble();
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getDouble(index));
-	}
-
-	@Test
-	public void testFloatTuple() {
-		final Float data = this.r.nextFloat();
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getFloat(index));
-	}
-
-	@Test
-	public void testIntegerTuple() {
-		final Integer data = this.r.nextInt();
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getInteger(index));
-	}
-
-	@Test
-	public void testLongTuple() {
-		final Long data = this.r.nextLong();
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getLong(index));
-	}
-
-	@Test
-	public void testShortTuple() {
-		final Short data = (short) this.r.nextInt();
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getShort(index));
-	}
-
-	@Test
-	public void testStringTuple() {
-		final byte[] rawdata = new byte[this.r.nextInt(15)];
-		this.r.nextBytes(rawdata);
-		final String data = new String(rawdata);
-
-		final int index = this.r.nextInt(5);
-		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
-		flinkTuple.setField(data, index);
-
-		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
-		Assert.assertEquals(flinkTuple.getField(index), tuple.getString(index));
-	}
-
-	@Test
-	public void testContains() throws Exception {
-		Fields schema = new Fields("a1", "a2");
-		StormTuple<Object> tuple = new StormTuple<Object>(Tuple.getTupleClass(1).newInstance(),
-				schema);
-
-		Assert.assertTrue(tuple.contains("a1"));
-		Assert.assertTrue(tuple.contains("a2"));
-		Assert.assertFalse(tuple.contains("a3"));
-	}
-
-	@Test
-	public void testGetFields() throws Exception {
-		Fields schema = new Fields();
-
-		Assert.assertSame(schema, new StormTuple<Object>(Tuple.getTupleClass(1).newInstance(),
-				schema).getFields());
-
-		Assert.assertSame(null, new StormTuple<Object>(null, schema).getFields());
-
-	}
-
-	@Test
-	public void testFieldIndex() throws Exception {
-		Fields schema = new Fields("a1", "a2");
-		StormTuple<Object> tuple = new StormTuple<Object>(Tuple.getTupleClass(1).newInstance(),
-				schema);
-
-		Assert.assertEquals(0, tuple.fieldIndex("a1"));
-		Assert.assertEquals(1, tuple.fieldIndex("a2"));
-	}
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testSelect() throws Exception {
-		Tuple tuple = Tuple.getTupleClass(arity).newInstance();
-		Values values = new Values();
-
-		ArrayList<String> attributeNames = new ArrayList<String>(arity);
-		ArrayList<String> filterNames = new ArrayList<String>(arity);
-
-		for (int i = 0; i < arity; ++i) {
-			tuple.setField(i, i);
-			attributeNames.add("a" + i);
-
-			if (r.nextBoolean()) {
-				filterNames.add("a" + i);
-				values.add(i);
-			}
-		}
-		Fields schema = new Fields(attributeNames);
-		Fields selector = new Fields(filterNames);
-
-		Assert.assertEquals(values, new StormTuple(tuple, schema).select(selector));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetValueByField() throws Exception {
-		Object value = mock(Object.class);
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getValueByField(fieldName));
-
-	}
-
-	@Test
-	public void testGetValueByFieldPojo() throws Exception {
-		Object value = mock(Object.class);
-		TestPojoMember<Object> pojo = new TestPojoMember<Object>(value);
-		StormTuple<TestPojoMember<Object>> tuple = new StormTuple<TestPojoMember<Object>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getValueByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetValueByFieldPojoGetter() throws Exception {
-		Object value = mock(Object.class);
-		TestPojoGetter<Object> pojo = new TestPojoGetter<Object>(value);
-		StormTuple<TestPojoGetter<Object>> tuple = new StormTuple<TestPojoGetter<Object>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getValueByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetStringByField() throws Exception {
-		String value = "stringValue";
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getStringByField(fieldName));
-	}
-
-	@Test
-	public void testGetStringByFieldPojo() throws Exception {
-		String value = "stringValue";
-		TestPojoMember<String> pojo = new TestPojoMember<String>(value);
-		StormTuple<TestPojoMember<String>> tuple = new StormTuple<TestPojoMember<String>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getStringByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetStringByFieldPojoGetter() throws Exception {
-		String value = "stringValue";
-		TestPojoGetter<String> pojo = new TestPojoGetter<String>(value);
-		StormTuple<TestPojoGetter<String>> tuple = new StormTuple<TestPojoGetter<String>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getStringByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetIntegerByField() throws Exception {
-		Integer value = r.nextInt();
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getIntegerByField(fieldName));
-	}
-
-	@Test
-	public void testGetIntegerByFieldPojo() throws Exception {
-		Integer value = r.nextInt();
-		TestPojoMember<Integer> pojo = new TestPojoMember<Integer>(value);
-		StormTuple<TestPojoMember<Integer>> tuple = new StormTuple<TestPojoMember<Integer>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getIntegerByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetIntegerByFieldPojoGetter() throws Exception {
-		Integer value = r.nextInt();
-		TestPojoGetter<Integer> pojo = new TestPojoGetter<Integer>(value);
-		StormTuple<TestPojoGetter<Integer>> tuple = new StormTuple<TestPojoGetter<Integer>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getIntegerByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetLongByField() throws Exception {
-		Long value = r.nextLong();
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getLongByField(fieldName));
-	}
-
-	@Test
-	public void testGetLongByFieldPojo() throws Exception {
-		Long value = r.nextLong();
-		TestPojoMember<Long> pojo = new TestPojoMember<Long>(value);
-		StormTuple<TestPojoMember<Long>> tuple = new StormTuple<TestPojoMember<Long>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getLongByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetLongByFieldPojoGetter() throws Exception {
-		Long value = r.nextLong();
-		TestPojoGetter<Long> pojo = new TestPojoGetter<Long>(value);
-		StormTuple<TestPojoGetter<Long>> tuple = new StormTuple<TestPojoGetter<Long>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getLongByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetBooleanByField() throws Exception {
-		Boolean value = r.nextBoolean();
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertEquals(value, tuple.getBooleanByField(fieldName));
-	}
-
-	@Test
-	public void testGetBooleanByFieldPojo() throws Exception {
-		Boolean value = r.nextBoolean();
-		TestPojoMember<Boolean> pojo = new TestPojoMember<Boolean>(value);
-		StormTuple<TestPojoMember<Boolean>> tuple = new StormTuple<TestPojoMember<Boolean>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getBooleanByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetBooleanByFieldPojoGetter() throws Exception {
-		Boolean value = r.nextBoolean();
-		TestPojoGetter<Boolean> pojo = new TestPojoGetter<Boolean>(value);
-		StormTuple<TestPojoGetter<Boolean>> tuple = new StormTuple<TestPojoGetter<Boolean>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getBooleanByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetShortByField() throws Exception {
-		Short value = (short) r.nextInt();
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getShortByField(fieldName));
-	}
-
-	@Test
-	public void testGetShortByFieldPojo() throws Exception {
-		Short value = (short) r.nextInt();
-		TestPojoMember<Short> pojo = new TestPojoMember<Short>(value);
-		StormTuple<TestPojoMember<Short>> tuple = new StormTuple<TestPojoMember<Short>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getShortByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetShortByFieldPojoGetter() throws Exception {
-		Short value = (short) r.nextInt();
-		TestPojoGetter<Short> pojo = new TestPojoGetter<Short>(value);
-		StormTuple<TestPojoGetter<Short>> tuple = new StormTuple<TestPojoGetter<Short>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getShortByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetByteByField() throws Exception {
-		Byte value = new Byte((byte) r.nextInt());
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getByteByField(fieldName));
-	}
-
-	@Test
-	public void testGetByteByFieldPojo() throws Exception {
-		Byte value = new Byte((byte) r.nextInt());
-		TestPojoMember<Byte> pojo = new TestPojoMember<Byte>(value);
-		StormTuple<TestPojoMember<Byte>> tuple = new StormTuple<TestPojoMember<Byte>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getByteByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetByteByFieldPojoGetter() throws Exception {
-		Byte value = new Byte((byte) r.nextInt());
-		TestPojoGetter<Byte> pojo = new TestPojoGetter<Byte>(value);
-		StormTuple<TestPojoGetter<Byte>> tuple = new StormTuple<TestPojoGetter<Byte>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getByteByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetDoubleByField() throws Exception {
-		Double value = r.nextDouble();
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getDoubleByField(fieldName));
-	}
-
-	@Test
-	public void testGetDoubleByFieldPojo() throws Exception {
-		Double value = r.nextDouble();
-		TestPojoMember<Double> pojo = new TestPojoMember<Double>(value);
-		StormTuple<TestPojoMember<Double>> tuple = new StormTuple<TestPojoMember<Double>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getDoubleByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetDoubleByFieldPojoGetter() throws Exception {
-		Double value = r.nextDouble();
-		TestPojoGetter<Double> pojo = new TestPojoGetter<Double>(value);
-		StormTuple<TestPojoGetter<Double>> tuple = new StormTuple<TestPojoGetter<Double>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getDoubleByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetFloatByField() throws Exception {
-		Float value = r.nextFloat();
-		StormTuple tuple = testGetByField(arity, index, value);
-		Assert.assertSame(value, tuple.getFloatByField(fieldName));
-	}
-
-	@Test
-	public void testGetFloatByFieldPojo() throws Exception {
-		Float value = r.nextFloat();
-		TestPojoMember<Float> pojo = new TestPojoMember<Float>(value);
-		StormTuple<TestPojoMember<Float>> tuple = new StormTuple<TestPojoMember<Float>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getFloatByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetFloatByFieldPojoGetter() throws Exception {
-		Float value = r.nextFloat();
-		TestPojoGetter<Float> pojo = new TestPojoGetter<Float>(value);
-		StormTuple<TestPojoGetter<Float>> tuple = new StormTuple<TestPojoGetter<Float>>(pojo,
-				null);
-		Assert.assertSame(value, tuple.getFloatByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Test
-	public void testGetBinaryByField() throws Exception {
-		byte[] data = new byte[1 + r.nextInt(20)];
-		r.nextBytes(data);
-		StormTuple tuple = testGetByField(arity, index, data);
-		Assert.assertSame(data, tuple.getBinaryByField(fieldName));
-	}
-
-	@Test
-	public void testGetBinaryFieldPojo() throws Exception {
-		byte[] data = new byte[1 + r.nextInt(20)];
-		r.nextBytes(data);
-		TestPojoMember<byte[]> pojo = new TestPojoMember<byte[]>(data);
-		StormTuple<TestPojoMember<byte[]>> tuple = new StormTuple<TestPojoMember<byte[]>>(pojo,
-				null);
-		Assert.assertSame(data, tuple.getBinaryByField(fieldNamePojo));
-	}
-
-	@Test
-	public void testGetBinaryByFieldPojoGetter() throws Exception {
-		byte[] data = new byte[1 + r.nextInt(20)];
-		r.nextBytes(data);
-		TestPojoGetter<byte[]> pojo = new TestPojoGetter<byte[]>(data);
-		StormTuple<TestPojoGetter<byte[]>> tuple = new StormTuple<TestPojoGetter<byte[]>>(pojo,
-				null);
-		Assert.assertSame(data, tuple.getBinaryByField(fieldNamePojo));
-	}
-
-	@SuppressWarnings({ "unchecked", "rawtypes" })
-	private <T> StormTuple testGetByField(int arity, int index, T value)
-			throws Exception {
-
-		assert (index < arity);
-
-		Tuple tuple = Tuple.getTupleClass(arity).newInstance();
-		tuple.setField(value, index);
-
-		ArrayList<String> attributeNames = new ArrayList<String>(arity);
-		for(int i = 0; i < arity; ++i) {
-			if(i == index) {
-				attributeNames.add(fieldName);
-			} else {
-				attributeNames.add("" + i);
-			}
-		}
-		Fields schema = new Fields(attributeNames);
-
-		return new StormTuple(tuple, schema);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testGetSourceGlobalStreamid() {
-		new StormTuple<Object>(null, null).getSourceGlobalStreamid();
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testGetSourceComponent() {
-		new StormTuple<Object>(null, null).getSourceComponent();
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testGetSourceTask() {
-		new StormTuple<Object>(null, null).getSourceTask();
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testGetSourceStreamId() {
-		new StormTuple<Object>(null, null).getSourceStreamId();
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testGetMessageId() {
-		new StormTuple<Object>(null, null).getMessageId();
-	}
-
-	public static class TestPojoMember<T> {
-		public T member;
-
-		public TestPojoMember(T value) {
-			this.member = value;
-		}
-	}
-
-	public static class TestPojoGetter<T> {
-		private T member;
-
-		public TestPojoGetter(T value) {
-			this.member = value;
-		}
-
-		public T getMember() {
-			return this.member;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java
----------------------------------------------------------------------
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
deleted file mode 100644
index c799d63..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java
+++ /dev/null
@@ -1,315 +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.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 {
-
-	@Test
-	public void testEmptyDeclarerBolt() {
-		IComponent boltOrSpout;
-
-		if (this.r.nextBoolean()) {
-			boltOrSpout = mock(IRichSpout.class);
-		} else {
-			boltOrSpout = mock(IRichBolt.class);
-		}
-
-		Assert.assertEquals(new HashMap<String, Integer>(),
-				StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null));
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testRawType() throws Exception {
-		IComponent boltOrSpout;
-
-		if (this.r.nextBoolean()) {
-			boltOrSpout = mock(IRichSpout.class);
-		} else {
-			boltOrSpout = mock(IRichBolt.class);
-		}
-
-		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-		declarer.declare(new Fields("dummy1", "dummy2"));
-		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout,
-				Sets.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }));
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testToManyAttributes() throws Exception {
-		IComponent boltOrSpout;
-
-		if (this.r.nextBoolean()) {
-			boltOrSpout = mock(IRichSpout.class);
-		} else {
-			boltOrSpout = mock(IRichBolt.class);
-		}
-
-		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(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		StormWrapperSetupHelper.getNumberOfAttributes(boltOrSpout, null);
-	}
-
-	@Test
-	public void testTupleTypes() throws Exception {
-		for (int i = -1; i < 26; ++i) {
-			this.testTupleTypes(i);
-		}
-	}
-
-	private void testTupleTypes(final int numberOfAttributes) throws Exception {
-		String[] schema;
-		if (numberOfAttributes == -1) {
-			schema = new String[1];
-		} else {
-			schema = new String[numberOfAttributes];
-		}
-		for (int i = 0; i < schema.length; ++i) {
-			schema[i] = "a" + i;
-		}
-
-		IComponent boltOrSpout;
-		if (this.r.nextBoolean()) {
-			boltOrSpout = mock(IRichSpout.class);
-		} else {
-			boltOrSpout = mock(IRichBolt.class);
-		}
-
-		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-		declarer.declare(new Fields(schema));
-		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
-
-		HashMap<String, Integer> attributes = new HashMap<String, Integer>();
-		attributes.put(Utils.DEFAULT_STREAM_ID, numberOfAttributes);
-
-		Assert.assertEquals(attributes, StormWrapperSetupHelper.getNumberOfAttributes(
-				boltOrSpout,
-				numberOfAttributes == -1 ? Sets
-						.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }) : null));
-	}
-
-	@Test
-	public void testCreateTopologyContext() {
-		HashMap<String, Integer> dops = new HashMap<String, Integer>();
-		dops.put("spout1", 1);
-		dops.put("spout2", 3);
-		dops.put("bolt1", 1);
-		dops.put("bolt2", 2);
-		dops.put("sink", 1);
-
-		HashMap<String, Integer> taskCounter = new HashMap<String, Integer>();
-		taskCounter.put("spout1", 0);
-		taskCounter.put("spout2", 0);
-		taskCounter.put("bolt1", 0);
-		taskCounter.put("bolt2", 0);
-		taskCounter.put("sink", 0);
-
-		HashMap<String, IComponent> operators = new HashMap<String, IComponent>();
-		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<Integer> taskIds = new HashSet<Integer>();
-
-		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<Integer, String> taskToComponents = new HashMap<Integer, String>();
-			Set<Integer> allTaskIds = new HashSet<Integer>();
-			for (String componentId : expectedContext.getComponentIds()) {
-				List<Integer> possibleTasks = expectedContext.getComponentTasks(componentId);
-				List<Integer> tasks = topologyContext.getComponentTasks(componentId);
-
-				Iterator<Integer> p_it = possibleTasks.iterator();
-				Iterator<Integer> 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 */ }
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java
deleted file mode 100644
index 7c91e6f..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/TestContext.java
+++ /dev/null
@@ -1,56 +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.java.tuple.Tuple1;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-
-import java.util.LinkedList;
-
-class TestContext implements SourceContext<Tuple1<Integer>> {
-	public LinkedList<Tuple1<Integer>> result = new LinkedList<Tuple1<Integer>>();
-
-	public TestContext() {
-	}
-
-	@Override
-	public void collect(final Tuple1<Integer> record) {
-		this.result.add(record.copy());
-	}
-
-	@Override
-	public void collectWithTimestamp(Tuple1<Integer> element, long timestamp) {
-		this.result.add(element.copy());
-	}
-
-	@Override
-	public void emitWatermark(Watermark mark) {
-		// ignore it
-	}
-
-	@Override
-	public Object getCheckpointLock() {
-		return null;
-	}
-
-	@Override
-	public void close() {
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j-test.properties b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j-test.properties
deleted file mode 100644
index 0b686e5..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,27 +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.
-################################################################################
-
-# Set root logger level to DEBUG and its only appender to A1.
-log4j.rootLogger=OFF, A1
-
-# A1 is set to be a ConsoleAppender.
-log4j.appender.A1=org.apache.log4j.ConsoleAppender
-
-# A1 uses PatternLayout.
-log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j.properties b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j.properties
deleted file mode 100644
index ed2bbcb..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,27 +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.
-################################################################################
-
-# This file ensures that tests executed from the IDE show log output
-
-log4j.rootLogger=OFF, console
-
-# Log all infos in the given file
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target = System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/logback-test.xml b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/logback-test.xml
deleted file mode 100644
index 4f56748..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +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.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/README.md
----------------------------------------------------------------------
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 6290df2..0000000
--- 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.


[15/15] flink git commit: [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 cle

Posted by mj...@apache.org.
[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


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7a67a60f
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7a67a60f
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7a67a60f

Branch: refs/heads/master
Commit: 7a67a60f934123863ca96a95e30471c99bb8088a
Parents: 39115ab
Author: mjsax <mj...@informatik.hu-berlin.de>
Authored: Tue Sep 15 23:59:31 2015 +0200
Committer: mjsax <mj...@informatik.hu-berlin.de>
Committed: Tue Oct 6 13:29:32 2015 +0200

----------------------------------------------------------------------
 .../flink-storm-compatibility-core/README.md    |   2 +-
 .../api/FlinkOutputFieldsDeclarer.java          | 170 ----------------
 .../stormcompatibility/api/FlinkTopology.java   |  16 +-
 .../api/FlinkTopologyBuilder.java               |  39 ++--
 .../api/FlinkTopologyContext.java               | 161 ---------------
 .../util/FiniteStormSpout.java                  |  39 ++++
 .../util/FlinkOutputFieldsDeclarer.java         | 168 ++++++++++++++++
 .../util/FlinkStormStreamSelector.java          |   2 +-
 .../util/FlinkTopologyContext.java              | 164 ++++++++++++++++
 .../util/SplitStreamTypeKeySelector.java        |  46 +++++
 .../wrappers/AbstractStormSpoutWrapper.java     |  36 ++--
 .../wrappers/FiniteStormSpout.java              |  37 ----
 .../wrappers/FiniteStormSpoutWrapper.java       |   1 +
 .../wrappers/SetupOutputFieldsDeclarer.java     |  63 ++++++
 .../wrappers/StormBoltWrapper.java              |  17 +-
 .../wrappers/StormOutputFieldsDeclarer.java     |  63 ------
 .../wrappers/StormWrapperSetupHelper.java       | 192 ++++++++++++++++--
 .../api/FlinkOutputFieldsDeclarerTest.java      | 193 ------------------
 .../api/FlinkTopologyContextTest.java           |  74 -------
 .../api/FlinkTopologyTest.java                  |  10 +-
 .../api/TestTopologyBuilder.java                |  27 +++
 .../util/FiniteTestSpout.java                   |  77 ++++++++
 .../util/FlinkOutputFieldsDeclarerTest.java     | 193 ++++++++++++++++++
 .../util/FlinkStormStreamSelectorTest.java      |  51 +++++
 .../util/FlinkTopologyContextTest.java          | 114 +++++++++++
 .../stormcompatibility/util/TestDummyBolt.java  |  20 +-
 .../stormcompatibility/util/TestDummySpout.java |  17 +-
 .../flink/stormcompatibility/util/TestSink.java |  16 +-
 .../wrappers/FiniteStormSpoutWrapperTest.java   |   6 +
 .../wrappers/FiniteTestSpout.java               |  77 --------
 .../wrappers/FlinkStormStreamSelectorTest.java  |  51 -----
 .../wrappers/SetupOutputFieldsDeclarerTest.java |  91 +++++++++
 .../wrappers/StormBoltWrapperTest.java          |  36 ++--
 .../wrappers/StormFiniteSpoutWrapperTest.java   |  13 +-
 .../wrappers/StormOutputFieldsDeclarerTest.java |  91 ---------
 .../wrappers/StormSpoutWrapperTest.java         |   6 +
 .../wrappers/StormWrapperSetupHelperTest.java   | 194 ++++++++++++++++++-
 .../util/FiniteStormFileSpout.java              |   2 +-
 .../util/FiniteStormInMemorySpout.java          |   2 +-
 .../split/SplitBoltTopology.java                |   4 +-
 40 files changed, 1565 insertions(+), 1016 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/README.md
----------------------------------------------------------------------
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 f42dc24..9663fc7 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

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java
----------------------------------------------------------------------
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/api/FlinkOutputFieldsDeclarer.java
deleted file mode 100644
index e2f6332..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarer.java
+++ /dev/null
@@ -1,170 +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.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.utils.Utils;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-
-import java.util.HashMap;
-import java.util.List;
-
-/**
- * {@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}.<br />
- * <br />
- * <strong>CAUTION: Currently, Flink does only support the default output stream. Furthermore,
- * direct emit is not supported.</strong>
- */
-final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer {
-
-	/** the declared output streams and schemas */
-	final HashMap<String, Fields> outputStreams = new HashMap<String, Fields>();
-
-	@Override
-	public void declare(final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
-	}
-
-	/**
-	 * {@inheritDoc}
-	 * <p/>
-	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		if {@code direct} is {@code true}
-	 */
-	@Override
-	public void declare(final boolean direct, final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
-	}
-
-	@Override
-	public void declareStream(final String streamId, final Fields fields) {
-		this.declareStream(streamId, false, fields);
-	}
-
-	/**
-	 * {@inheritDoc}
-	 * <p/>
-	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		if {@code direct} is {@code true}
-	 */
-	@Override
-	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
-		if (direct) {
-			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
-		}
-
-		this.outputStreams.put(streamId, fields);
-	}
-
-	/**
-	 * Returns {@link TypeInformation} for the declared output schema for a specific stream.
-	 * 
-	 * @param streamId
-	 *            A stream ID.
-	 * 
-	 * @return output type information for the declared output schema of the specified stream; or {@code null} if
-	 *         {@code streamId == null}
-	 * 
-	 * @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 {
-		if (streamId == null) {
-			return null;
-		}
-
-		Fields outputSchema = this.outputStreams.get(streamId);
-		if (outputSchema == null) {
-			throw new IllegalArgumentException("Stream with ID '" + streamId
-					+ "' was not declared.");
-		}
-
-		Tuple t;
-		final int numberOfAttributes = outputSchema.size();
-
-		if (numberOfAttributes == 1) {
-			return TypeExtractor.getForClass(Object.class);
-		} else if (numberOfAttributes <= 25) {
-			try {
-				t = Tuple.getTupleClass(numberOfAttributes).newInstance();
-			} catch (final InstantiationException e) {
-				throw new RuntimeException(e);
-			} catch (final IllegalAccessException e) {
-				throw new RuntimeException(e);
-			}
-		} else {
-			throw new IllegalArgumentException("Flink supports only a maximum number of 25 attributes");
-		}
-
-		// TODO: declare only key fields as DefaultComparable
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			t.setField(new DefaultComparable(), i);
-		}
-
-		return TypeExtractor.getForObject(t);
-	}
-
-	/**
-	 * {@link DefaultComparable} is a {@link Comparable} helper class that is used to get the correct {@link
-	 * TypeInformation} from {@link TypeExtractor} within {@link #getOutputType()}. If key fields are not comparable,
-	 * Flink cannot use them and will throw an exception.
-	 */
-	private static class DefaultComparable implements Comparable<DefaultComparable> {
-
-		public DefaultComparable() {
-		}
-
-		@Override
-		public int compareTo(final DefaultComparable o) {
-			return 0;
-		}
-	}
-
-	/**
-	 * Computes the indexes within the declared output schema of the specified stream, for a list of given
-	 * field-grouping attributes.
-	 * 
-	 * @param streamId
-	 *            A stream ID.
-	 * @param groupingFields
-	 *            The names of the key fields.
-	 * 
-	 * @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<String> groupingFields) {
-		final int[] fieldIndexes = new int[groupingFields.size()];
-
-		for (int i = 0; i < fieldIndexes.length; ++i) {
-			fieldIndexes[i] = this.outputStreams.get(streamId).fieldIndex(groupingFields.get(i));
-		}
-
-		return fieldIndexes;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopology.java
----------------------------------------------------------------------
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 8c75a2c..179466e 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 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  * 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 @@ class FlinkTopology extends StreamExecutionEnvironment {
 	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 @@ class FlinkTopology extends StreamExecutionEnvironment {
 	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.");
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilder.java
----------------------------------------------------------------------
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 e4f6c94..d62d56b 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 backtype.storm.tuple.Fields;
 
 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<String, HashMap<String, Fields>> outputStreams = new HashMap<String, HashMap<String, Fields>>();
 	/** All spouts&bolts declarers by their ID */
 	private final HashMap<String, FlinkOutputFieldsDeclarer> declarers = new HashMap<String, FlinkOutputFieldsDeclarer>();
+	// 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<String, HashMap<String, DataStream>> availableInputs = new HashMap<String, HashMap<String, DataStream>>();
@@ -102,6 +107,7 @@ public class FlinkTopologyBuilder {
 			} else {
 				spoutWrapper = new StormSpoutWrapper(userSpout);
 			}
+			spoutWrapper.setStormTopology(stormTopology);
 
 			DataStreamSource source;
 			HashMap<String, DataStream> outputStreams = new HashMap<String, DataStream>();
@@ -126,6 +132,8 @@ public class FlinkTopologyBuilder {
 			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 class FlinkTopologyBuilder {
 							}
 
 							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 class FlinkTopologyBuilder {
 								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 class FlinkTopologyBuilder {
 								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 class FlinkTopologyBuilder {
 								}
 								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 class FlinkTopologyBuilder {
 	 * }
 	 */
 
+	// for internal testing purpose only
+	StormTopology getStormTopology() {
+		return this.stormTopology;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContext.java
----------------------------------------------------------------------
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/api/FlinkTopologyContext.java
deleted file mode 100644
index a761617..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContext.java
+++ /dev/null
@@ -1,161 +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.generated.StormTopology;
-import backtype.storm.hooks.ITaskHook;
-import backtype.storm.metric.api.CombinedMetric;
-import backtype.storm.metric.api.ICombiner;
-import backtype.storm.metric.api.IMetric;
-import backtype.storm.metric.api.IReducer;
-import backtype.storm.metric.api.ReducedMetric;
-import backtype.storm.state.ISubscribedState;
-import backtype.storm.task.TopologyContext;
-
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * {@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 {
-
-	/**
-	 * 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<Integer, String> taskToComponents,
-			final Integer taskId) {
-		super(topology, null, taskToComponents, null, null, null, null, null, taskId, null, null, null, null, null,
-				null, null);
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public void addTaskHook(final ITaskHook hook) {
-		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public Collection<ITaskHook> getHooks() {
-		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public IMetric getRegisteredMetricByName(final String name) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@SuppressWarnings("rawtypes")
-	@Override
-	public CombinedMetric registerMetric(final String name, final ICombiner combiner, final int timeBucketSizeInSecs) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@SuppressWarnings("rawtypes")
-	@Override
-	public ReducedMetric registerMetric(final String name, final IReducer combiner, final int timeBucketSizeInSecs) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@SuppressWarnings("unchecked")
-	@Override
-	public IMetric registerMetric(final String name, final IMetric metric, final int timeBucketSizeInSecs) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public <T extends ISubscribedState> T setAllSubscribedState(final T obj) {
-		throw new UnsupportedOperationException("Not supported by Flink");
-
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final T obj) {
-		throw new UnsupportedOperationException("Not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final String streamId, final T
-			obj) {
-		throw new UnsupportedOperationException("Not supported by Flink");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java
new file mode 100644
index 0000000..114fa7c
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java
@@ -0,0 +1,39 @@
+/*
+ * 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.stormcompatibility.wrappers.FiniteStormSpoutWrapper;
+
+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}.
+ */
+public interface FiniteStormSpout extends IRichSpout {
+
+	/**
+	 * When returns true, the spout has reached the end of the stream.
+	 *
+	 * @return true, if the spout's stream reached its end, false otherwise
+	 */
+	public boolean reachedEnd();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java
new file mode 100644
index 0000000..3eee8d6
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java
@@ -0,0 +1,168 @@
+/*
+ * 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 backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * {@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}.<br />
+ * <br />
+ * <strong>CAUTION: Flink does not support direct emit.</strong>
+ */
+public final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer {
+
+	/** The declared output streams and schemas. */
+	public final HashMap<String, Fields> outputStreams = new HashMap<String, Fields>();
+
+	@Override
+	public void declare(final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
+	}
+
+	/**
+	 * {@inheritDoc}
+	 * <p/>
+	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		if {@code direct} is {@code true}
+	 */
+	@Override
+	public void declare(final boolean direct, final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
+	}
+
+	@Override
+	public void declareStream(final String streamId, final Fields fields) {
+		this.declareStream(streamId, false, fields);
+	}
+
+	/**
+	 * {@inheritDoc}
+	 * <p/>
+	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		if {@code direct} is {@code true}
+	 */
+	@Override
+	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
+		if (direct) {
+			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
+		}
+
+		this.outputStreams.put(streamId, fields);
+	}
+
+	/**
+	 * Returns {@link TypeInformation} for the declared output schema for a specific stream.
+	 * 
+	 * @param streamId
+	 *            A stream ID.
+	 * 
+	 * @return output type information for the declared output schema of the specified stream; or {@code null} if
+	 *         {@code streamId == null}
+	 * 
+	 * @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 {
+		if (streamId == null) {
+			return null;
+		}
+
+		Fields outputSchema = this.outputStreams.get(streamId);
+		if (outputSchema == null) {
+			throw new IllegalArgumentException("Stream with ID '" + streamId
+					+ "' was not declared.");
+		}
+
+		Tuple t;
+		final int numberOfAttributes = outputSchema.size();
+
+		if (numberOfAttributes == 1) {
+			return TypeExtractor.getForClass(Object.class);
+		} else if (numberOfAttributes <= 25) {
+			try {
+				t = Tuple.getTupleClass(numberOfAttributes).newInstance();
+			} catch (final InstantiationException e) {
+				throw new RuntimeException(e);
+			} catch (final IllegalAccessException e) {
+				throw new RuntimeException(e);
+			}
+		} else {
+			throw new IllegalArgumentException("Flink supports only a maximum number of 25 attributes");
+		}
+
+		// TODO: declare only key fields as DefaultComparable
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			t.setField(new DefaultComparable(), i);
+		}
+
+		return TypeExtractor.getForObject(t);
+	}
+
+	/**
+	 * {@link DefaultComparable} is a {@link Comparable} helper class that is used to get the correct {@link
+	 * TypeInformation} from {@link TypeExtractor} within {@link #getOutputType()}. If key fields are not comparable,
+	 * Flink cannot use them and will throw an exception.
+	 */
+	private static class DefaultComparable implements Comparable<DefaultComparable> {
+
+		public DefaultComparable() {
+		}
+
+		@Override
+		public int compareTo(final DefaultComparable o) {
+			return 0;
+		}
+	}
+
+	/**
+	 * Computes the indexes within the declared output schema of the specified stream, for a list of given
+	 * field-grouping attributes.
+	 * 
+	 * @param streamId
+	 *            A stream ID.
+	 * @param groupingFields
+	 *            The names of the key fields.
+	 * 
+	 * @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<String> groupingFields) {
+		final int[] fieldIndexes = new int[groupingFields.size()];
+
+		for (int i = 0; i < fieldIndexes.length; ++i) {
+			fieldIndexes[i] = this.outputStreams.get(streamId).fieldIndex(groupingFields.get(i));
+		}
+
+		return fieldIndexes;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java
----------------------------------------------------------------------
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 7ca45d6..7e60a87 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 @@ import org.apache.flink.streaming.api.collector.selector.OutputSelector;
 /**
  * Used by {@link FlinkTopologyBuilder} to split multiple declared output streams within Flink.
  */
-final public class FlinkStormStreamSelector<T> implements OutputSelector<SplitStreamType<T>> {
+public final class FlinkStormStreamSelector<T> implements OutputSelector<SplitStreamType<T>> {
 	private static final long serialVersionUID = 2553423379715401023L;
 
 	/** internal cache to avoid short living ArrayList objects. */

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java
new file mode 100644
index 0000000..14af830
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java
@@ -0,0 +1,164 @@
+/*
+ * 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 backtype.storm.generated.StormTopology;
+import backtype.storm.hooks.ITaskHook;
+import backtype.storm.metric.api.CombinedMetric;
+import backtype.storm.metric.api.ICombiner;
+import backtype.storm.metric.api.IMetric;
+import backtype.storm.metric.api.IReducer;
+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 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,
+			final Map<Integer, String> taskToComponent, final Map<String, List<Integer>> componentToSortedTasks,
+			final Map<String, Map<String, Fields>> componentToStreamToFields, final String stormId, final String codeDir,
+			final String pidDir, final Integer taskId, final Integer workerPort, final List<Integer> workerTasks,
+			final Map<String, Object> defaultResources, final Map<String, Object> userResources,
+			final Map<String, Object> 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);
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public void addTaskHook(final ITaskHook hook) {
+		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public Collection<ITaskHook> getHooks() {
+		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public IMetric getRegisteredMetricByName(final String name) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@SuppressWarnings("rawtypes")
+	@Override
+	public CombinedMetric registerMetric(final String name, final ICombiner combiner, final int timeBucketSizeInSecs) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@SuppressWarnings("rawtypes")
+	@Override
+	public ReducedMetric registerMetric(final String name, final IReducer combiner, final int timeBucketSizeInSecs) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@SuppressWarnings("unchecked")
+	@Override
+	public IMetric registerMetric(final String name, final IMetric metric, final int timeBucketSizeInSecs) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public <T extends ISubscribedState> T setAllSubscribedState(final T obj) {
+		throw new UnsupportedOperationException("Not supported by Flink");
+
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final T obj) {
+		throw new UnsupportedOperationException("Not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final String streamId, final T
+			obj) {
+		throw new UnsupportedOperationException("Not supported by Flink");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java
----------------------------------------------------------------------
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 0000000..200f772
--- /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<SplitStreamType<Tuple>, Tuple> {
+	private static final long serialVersionUID = 4672434660037669254L;
+
+	private final ArrayKeySelector<Tuple> selector;
+
+	public SplitStreamTypeKeySelector(int... fields) {
+		this.selector = new KeySelectorUtil.ArrayKeySelector<Tuple>(fields);
+	}
+
+	@Override
+	public Tuple getKey(SplitStreamType<Tuple> value) throws Exception {
+		return selector.getKey(value.value);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java
----------------------------------------------------------------------
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 c531580..ccd29bb 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 @@ 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;
 
@@ -43,22 +44,16 @@ import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
 public abstract class AbstractStormSpoutWrapper<OUT> extends RichParallelSourceFunction<OUT> {
 	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<String, Integer> 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<OUT> 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 abstract class AbstractStormSpoutWrapper<OUT> extends RichParallelSourceF
 		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<OUT> ctx) throws Exception {
 		this.collector = new StormSpoutCollector<OUT>(this.numberOfAttributes, ctx);
@@ -114,8 +119,11 @@ public abstract class AbstractStormSpoutWrapper<OUT> extends RichParallelSourceF
 		}
 
 		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();

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java
----------------------------------------------------------------------
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/wrappers/FiniteStormSpout.java
deleted file mode 100644
index 58a4f7a..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpout.java
+++ /dev/null
@@ -1,37 +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;
-
-/**
- * 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}.
- */
-public interface FiniteStormSpout extends IRichSpout {
-
-	/**
-	 * When returns true, the spout has reached the end of the stream.
-	 *
-	 * @return true, if the spout's stream reached its end, false otherwise
-	 */
-	public boolean reachedEnd();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java
----------------------------------------------------------------------
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 1912afc..f499ecc 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 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;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java
new file mode 100644
index 0000000..3cd27d4
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java
@@ -0,0 +1,63 @@
+/*
+ * 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.HashMap;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+/**
+ * {@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 SetupOutputFieldsDeclarer implements OutputFieldsDeclarer {
+
+	/** The number of attributes for each declared stream by the wrapped operator. */
+	HashMap<String, Integer> outputSchemas = new HashMap<String, Integer>();
+
+	@Override
+	public void declare(final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
+	}
+
+	@Override
+	public void declare(final boolean direct, final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
+	}
+
+	@Override
+	public void declareStream(final String streamId, final Fields fields) {
+		this.declareStream(streamId, false, fields);
+	}
+
+	@Override
+	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
+		if (streamId == null) {
+			throw new IllegalArgumentException("Stream ID cannot be null.");
+		}
+		if (direct) {
+			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
+		}
+
+		this.outputSchemas.put(streamId, fields.size());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java
----------------------------------------------------------------------
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 6b58b0a..715d6df 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 @@ package org.apache.flink.stormcompatibility.wrappers;
 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<IN, OUT> extends AbstractStreamOperator<OUT> imple
 	private final HashMap<String, Integer> 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 class StormBoltWrapper<IN, OUT> extends AbstractStreamOperator<OUT> imple
 		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<OUT>(output);
 		OutputCollector stormCollector = null;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java
----------------------------------------------------------------------
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/StormOutputFieldsDeclarer.java
deleted file mode 100644
index f33d4d3..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarer.java
+++ /dev/null
@@ -1,63 +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.HashMap;
-
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.utils.Utils;
-
-/**
- * {@link StormOutputFieldsDeclarer} 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 {
-
-	/** The number of attributes for each declared stream by the wrapped operator. */
-	HashMap<String, Integer> outputSchemas = new HashMap<String, Integer>();
-
-	@Override
-	public void declare(final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
-	}
-
-	@Override
-	public void declare(final boolean direct, final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
-	}
-
-	@Override
-	public void declareStream(final String streamId, final Fields fields) {
-		this.declareStream(streamId, false, fields);
-	}
-
-	@Override
-	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
-		if (streamId == null) {
-			throw new IllegalArgumentException("Stream ID cannot be null.");
-		}
-		if (direct) {
-			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
-		}
-
-		this.outputSchemas.put(streamId, fields.size());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java
----------------------------------------------------------------------
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 75ab8e0..891497e 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<String, Integer> getNumberOfAttributes(final IComponent spoutOrBolt,
 			final Collection<String> rawOutputs)
 					throws IllegalArgumentException {
-		final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
 		spoutOrBolt.declareOutputFields(declarer);
 
 		for (Entry<String, Integer> schema : declarer.outputSchemas.entrySet()) {
@@ -84,27 +95,174 @@ class StormWrapperSetupHelper {
 		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<Integer, String> taskToComponents = new HashMap<Integer, String>();
-		taskToComponents.put(taskId, context.getTaskName());
+		final Map<String, List<Integer>> componentToSortedTasks = new HashMap<String, List<Integer>>();
+		final Map<String, Map<String, Fields>> componentToStreamToFields = new HashMap<String, Map<String, Fields>>();
+		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<Integer> workerTasks = new ArrayList<Integer>();
+		final Map<String, Object> defaultResources = new HashMap<String, Object>();
+		final Map<String, Object> userResources = new HashMap<String, Object>();
+		final Map<String, Object> executorData = new HashMap<String, Object>();
+		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<String, Bolt> bolts = new HashMap<String, Bolt>();
-		final Map<String, SpoutSpec> spoutSpecs = new HashMap<String, SpoutSpec>();
+			HashMap<String, SpoutSpec> spouts = new HashMap<String, SpoutSpec>();
+			HashMap<String, Bolt> bolts = new HashMap<String, Bolt>();
+			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<String, StateSpoutSpec>());
 
-		if (spoutOrBolt) {
-			spoutSpecs.put(context.getTaskName(), new SpoutSpec(null, common));
+			taskId = context.getIndexOfThisSubtask();
+
+			List<Integer> sortedTasks = new ArrayList<Integer>(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<String, SpoutSpec> spouts = stormTopology.get_spouts();
+			Map<String, Bolt> bolts = stormTopology.get_bolts();
+			Map<String, StateSpoutSpec> stateSpouts = stormTopology.get_state_spouts();
+
+			tid = 1;
+
+			for (Entry<String, SpoutSpec> 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<String, Bolt> 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<String, StateSpoutSpec> 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<Integer, String> taskToComponents,
+			final Map<String, List<Integer>> componentToSortedTasks,
+			final Map<String, Map<String, Fields>> componentToStreamToFields) {
+		final int parallelism_hint = common.get_parallelism_hint();
+		Integer taskId = null;
+
+		if (componentId.equals(operatorName)) {
+			taskId = tid + index;
+		}
+
+		List<Integer> sortedTasks = new ArrayList<Integer>(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<String, Fields> outputStreams = new HashMap<String, Fields>();
+		for(Entry<String, StreamInfo> 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;
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
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/api/FlinkOutputFieldsDeclarerTest.java
deleted file mode 100644
index 08ac60b..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkOutputFieldsDeclarerTest.java
+++ /dev/null
@@ -1,193 +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.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.junit.Assert;
-import org.junit.Test;
-
-import java.util.LinkedList;
-
-public class FlinkOutputFieldsDeclarerTest extends AbstractTest {
-
-
-
-	@Test
-	public void testNull() {
-		Assert.assertNull(new FlinkOutputFieldsDeclarer().getOutputType(null));
-	}
-
-	@Test
-	public void testDeclare() {
-		for (int i = 0; i < 2; ++i) { // test case: simple / non-direct
-			for (int j = 1; j < 2; ++j) { // number of streams
-				for (int k = 0; k <= 25; ++k) { // number of attributes
-					this.runDeclareTest(i, j, k);
-				}
-			}
-		}
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareSimpleToManyAttributes() {
-		this.runDeclareTest(0, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareNonDirectToManyAttributes() {
-		this.runDeclareTest(1, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareDefaultStreamToManyAttributes() {
-		this.runDeclareTest(2, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareFullToManyAttributes() {
-		this.runDeclareTest(3, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	private void runDeclareTest(final int testCase, final int numberOfStreams,
-			final int numberOfAttributes) {
-		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-
-		String[] streams = null;
-		if (numberOfStreams > 1 || r.nextBoolean()) {
-			streams = new String[numberOfStreams];
-			for (int i = 0; i < numberOfStreams; ++i) {
-				streams[i] = "stream" + i;
-			}
-		}
-
-		final String[] attributes = new String[numberOfAttributes];
-		for (int i = 0; i < attributes.length; ++i) {
-			attributes[i] = "a" + i;
-		}
-
-		switch (testCase) {
-		case 0:
-			this.declareSimple(declarer, streams, attributes);
-			break;
-		default:
-			this.declareNonDirect(declarer, streams, attributes);
-		}
-
-		if (streams == null) {
-			streams = new String[] { Utils.DEFAULT_STREAM_ID };
-		}
-
-		for (String stream : streams) {
-			final TypeInformation<?> type = declarer.getOutputType(stream);
-
-			if (numberOfAttributes == 1) {
-				Assert.assertEquals(type.getClass(), GenericTypeInfo.class);
-				Assert.assertEquals(type.getTypeClass(), Object.class);
-			} else {
-				Assert.assertEquals(numberOfAttributes, type.getArity());
-				Assert.assertTrue(type.isTupleType());
-			}
-		}
-	}
-
-	private void declareSimple(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
-			final String[] attributes) {
-
-		if (streams != null) {
-			for (String stream : streams) {
-				declarer.declareStream(stream, new Fields(attributes));
-			}
-		} else {
-			declarer.declare(new Fields(attributes));
-		}
-	}
-
-	private void declareNonDirect(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
-			final String[] attributes) {
-
-		if (streams != null) {
-			for (String stream : streams) {
-				declarer.declareStream(stream, false, new Fields(attributes));
-			}
-		} else {
-			declarer.declare(false, new Fields(attributes));
-		}
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testUndeclared() {
-		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-		declarer.getOutputType("unknownStreamId");
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareDirect() {
-		new FlinkOutputFieldsDeclarer().declare(true, null);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareDirect2() {
-		new FlinkOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, null);
-	}
-
-	@Test
-	public void testGetGroupingFieldIndexes() {
-		final int numberOfAttributes = 5 + this.r.nextInt(21);
-		final String[] attributes = new String[numberOfAttributes];
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			attributes[i] = "a" + i;
-		}
-
-		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-		declarer.declare(new Fields(attributes));
-
-		final int numberOfKeys = 1 + this.r.nextInt(25);
-		final LinkedList<String> groupingFields = new LinkedList<String>();
-		final boolean[] indexes = new boolean[numberOfAttributes];
-
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			if (this.r.nextInt(26) < numberOfKeys) {
-				groupingFields.add(attributes[i]);
-				indexes[i] = true;
-			} else {
-				indexes[i] = false;
-			}
-		}
-
-		final int[] expectedResult = new int[groupingFields.size()];
-		int j = 0;
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			if (indexes[i]) {
-				expectedResult[j++] = i;
-			}
-		}
-
-		final int[] result = declarer.getGroupingFieldIndexes(Utils.DEFAULT_STREAM_ID,
-				groupingFields);
-
-		Assert.assertEquals(expectedResult.length, result.length);
-		for (int i = 0; i < expectedResult.length; ++i) {
-			Assert.assertEquals(expectedResult[i], result[i]);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyContextTest.java
----------------------------------------------------------------------
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 d214610..0000000
--- 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);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java
----------------------------------------------------------------------
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 f179919..c98c9a3 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 class FlinkTopologyTest {
 
 	@Test(expected = AssertionError.class)
 	public void testAssert() {
-		new FlinkTopology(null).increaseNumberOfTasks(0);
+		new FlinkTopology().increaseNumberOfTasks(0);
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java
----------------------------------------------------------------------
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 0000000..f664e58
--- /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();
+	}
+}


[04/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.java
new file mode 100644
index 0000000..39e7a25
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalITCase.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.storm.wordcount;
+
+import org.apache.flink.storm.util.StormTestBase;
+import org.apache.flink.storm.wordcount.WordCountLocal;
+import org.apache.flink.test.testdata.WordCountData;
+
+public class WordCountLocalITCase extends StormTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WordCountLocal.main(new String[]{this.textPath, this.resultPath});
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.java
new file mode 100644
index 0000000..78acfe5
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/WordCountLocalNamedITCase.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.storm.wordcount;
+
+import org.apache.flink.storm.util.StormTestBase;
+import org.apache.flink.storm.wordcount.WordCountLocalByName;
+import org.apache.flink.test.testdata.WordCountData;
+
+public class WordCountLocalNamedITCase extends StormTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		WordCountLocalByName.main(new String[] { this.textPath, this.resultPath });
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/resources/log4j-test.properties b/flink-contrib/flink-storm-examples/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..0b686e5
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/resources/log4j-test.properties
@@ -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.
+################################################################################
+
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=OFF, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/resources/log4j.properties b/flink-contrib/flink-storm-examples/src/test/resources/log4j.properties
new file mode 100644
index 0000000..ed2bbcb
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/resources/log4j.properties
@@ -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.
+################################################################################
+
+# This file ensures that tests executed from the IDE show log output
+
+log4j.rootLogger=OFF, console
+
+# Log all infos in the given file
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target = System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/resources/logback-test.xml b/flink-contrib/flink-storm-examples/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..4f56748
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/resources/logback-test.xml
@@ -0,0 +1,30 @@
+<!--
+  ~ 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.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/README.md
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/README.md b/flink-contrib/flink-storm/README.md
new file mode 100644
index 0000000..239780c
--- /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

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml
new file mode 100644
index 0000000..657b974
--- /dev/null
+++ b/flink-contrib/flink-storm/pom.xml
@@ -0,0 +1,114 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-contrib-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-storm</artifactId>
+	<name>flink-storm</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-core</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.storm</groupId>
+			<artifactId>storm-core</artifactId>
+			<version>0.9.4</version>
+			<exclusions>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>log4j-over-slf4j</artifactId>
+				</exclusion>
+				<exclusion>
+					<artifactId>logback-classic</artifactId>
+					<groupId>ch.qos.logback</groupId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+
+		<pluginManagement>
+			<plugins>
+				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+				<plugin>
+					<groupId>org.eclipse.m2e</groupId>
+					<artifactId>lifecycle-mapping</artifactId>
+					<version>1.0.0</version>
+					<configuration>
+						<lifecycleMappingMetadata>
+							<pluginExecutions>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-dependency-plugin</artifactId>
+										<versionRange>[2.9,)</versionRange>
+										<goals>
+											<goal>unpack</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+							</pluginExecutions>
+						</lifecycleMappingMetadata>
+					</configuration>
+				</plugin>
+			</plugins>
+		</pluginManagement>
+
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
new file mode 100644
index 0000000..5f0ee21
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java
@@ -0,0 +1,315 @@
+/*
+ * 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.api;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+import backtype.storm.Config;
+import backtype.storm.generated.AlreadyAliveException;
+import backtype.storm.generated.InvalidTopologyException;
+import backtype.storm.generated.KillOptions;
+import backtype.storm.generated.Nimbus;
+import backtype.storm.generated.NotAliveException;
+import backtype.storm.utils.NimbusClient;
+import backtype.storm.utils.Utils;
+
+import com.google.common.collect.Lists;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.client.program.Client;
+import org.apache.flink.client.program.JobWithJars;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.client.JobStatusMessage;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmanager.JobManager;
+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.storm.util.StormConfig;
+
+import scala.Some;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * {@link FlinkClient} mimics a Storm {@link NimbusClient} and {@link Nimbus}{@code .Client} at once, to interact with
+ * Flink's JobManager instead of Storm's Nimbus.
+ */
+public class FlinkClient {
+
+	/** The client's configuration */
+	private final Map<?,?> conf;
+	/** The jobmanager's host name */
+	private final String jobManagerHost;
+	/** The jobmanager's rpc port */
+	private final int jobManagerPort;
+	/** The user specified timeout in milliseconds */
+	private final String timeout;
+
+	// The following methods are derived from "backtype.storm.utils.NimbusClient"
+
+	/**
+	 * Instantiates a new {@link FlinkClient} for the given configuration, host name, and port. If values for {@link
+	 * Config#NIMBUS_HOST} and {@link Config#NIMBUS_THRIFT_PORT} of the given configuration are ignored.
+	 *
+	 * @param conf
+	 * 		A configuration.
+	 * @param host
+	 * 		The jobmanager's host name.
+	 * @param port
+	 * 		The jobmanager's rpc port.
+	 */
+	@SuppressWarnings("rawtypes")
+	public FlinkClient(final Map conf, final String host, final int port) {
+		this(conf, host, port, null);
+	}
+
+	/**
+	 * Instantiates a new {@link FlinkClient} for the given configuration, host name, and port. If values for {@link
+	 * Config#NIMBUS_HOST} and {@link Config#NIMBUS_THRIFT_PORT} of the given configuration are ignored.
+	 *
+	 * @param conf
+	 * 		A configuration.
+	 * @param host
+	 * 		The jobmanager's host name.
+	 * @param port
+	 * 		The jobmanager's rpc port.
+	 * @param timeout
+	 * 		Timeout
+	 */
+	@SuppressWarnings("rawtypes")
+	public FlinkClient(final Map conf, final String host, final int port, final Integer timeout) {
+		this.conf = conf;
+		this.jobManagerHost = host;
+		this.jobManagerPort = port;
+		if (timeout != null) {
+			this.timeout = timeout + " ms";
+		} else {
+			this.timeout = null;
+		}
+	}
+
+	/**
+	 * Returns a {@link FlinkClient} that uses the configured {@link Config#NIMBUS_HOST} and {@link
+	 * Config#NIMBUS_THRIFT_PORT} as JobManager address.
+	 *
+	 * @param conf
+	 * 		Configuration that contains the jobmanager's hostname and port.
+	 * @return A configured {@link FlinkClient}.
+	 */
+	@SuppressWarnings("rawtypes")
+	public static FlinkClient getConfiguredClient(final Map conf) {
+		final String nimbusHost = (String) conf.get(Config.NIMBUS_HOST);
+		final int nimbusPort = Utils.getInt(conf.get(Config.NIMBUS_THRIFT_PORT)).intValue();
+		return new FlinkClient(conf, nimbusHost, nimbusPort);
+	}
+
+	/**
+	 * Return a reference to itself.
+	 * <p/>
+	 * {@link FlinkClient} mimics both, {@link NimbusClient} and {@link Nimbus}{@code .Client}, at once.
+	 *
+	 * @return A reference to itself.
+	 */
+	public FlinkClient getClient() {
+		return this;
+	}
+
+	// The following methods are derived from "backtype.storm.generated.Nimubs.Client"
+
+	/**
+	 * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink does not support
+	 * uploading a jar file before hand. Jar files are always uploaded directly when a program is submitted.
+	 */
+	public void submitTopology(final String name, final String uploadedJarLocation, final FlinkTopology topology)
+			throws AlreadyAliveException, InvalidTopologyException {
+		this.submitTopologyWithOpts(name, uploadedJarLocation, topology);
+	}
+
+	/**
+	 * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink does not support
+	 * uploading a jar file before hand. Jar files are always uploaded directly when a program is submitted.
+	 */
+	public void submitTopologyWithOpts(final String name, final String uploadedJarLocation, final FlinkTopology
+			topology)
+					throws AlreadyAliveException, InvalidTopologyException {
+
+		if (this.getTopologyJobId(name) != null) {
+			throw new AlreadyAliveException();
+		}
+
+		final File uploadedJarFile = new File(uploadedJarLocation);
+		try {
+			JobWithJars.checkJarFile(uploadedJarFile);
+		} catch (final IOException e) {
+			throw new RuntimeException("Problem with jar file " + uploadedJarFile.getAbsolutePath(), e);
+		}
+
+		/* set storm configuration */
+		if (this.conf != null) {
+			topology.getConfig().setGlobalJobParameters(new StormConfig(this.conf));
+		}
+
+		final JobGraph jobGraph = topology.getStreamGraph().getJobGraph(name);
+		jobGraph.addJar(new Path(uploadedJarFile.getAbsolutePath()));
+
+		final Configuration configuration = jobGraph.getJobConfiguration();
+		configuration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerHost);
+		configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort);
+
+		final Client client;
+		try {
+			client = new Client(configuration);
+		} catch (IOException e) {
+			throw new RuntimeException("Could not establish a connection to the job manager", e);
+		}
+
+		try {
+			ClassLoader classLoader = JobWithJars.buildUserCodeClassLoader(
+					Lists.newArrayList(uploadedJarFile),
+					this.getClass().getClassLoader());
+			client.runDetached(jobGraph, classLoader);
+		} catch (final ProgramInvocationException e) {
+			throw new RuntimeException("Cannot execute job due to ProgramInvocationException", e);
+		}
+	}
+
+	public void killTopology(final String name) throws NotAliveException {
+		this.killTopologyWithOpts(name, null);
+	}
+
+	public void killTopologyWithOpts(final String name, final KillOptions options) throws NotAliveException {
+		final JobID jobId = this.getTopologyJobId(name);
+		if (jobId == null) {
+			throw new NotAliveException();
+		}
+
+		try {
+			final ActorRef jobManager = this.getJobManager();
+
+			if (options != null) {
+				try {
+					Thread.sleep(1000 * options.get_wait_secs());
+				} catch (final InterruptedException e) {
+					throw new RuntimeException(e);
+				}
+			}
+
+			final FiniteDuration askTimeout = this.getTimeout();
+			final Future<Object> response = Patterns.ask(jobManager, new CancelJob(jobId), new Timeout(askTimeout));
+			try {
+				Await.result(response, askTimeout);
+			} catch (final Exception e) {
+				throw new RuntimeException("Killing topology " + name + " with Flink job ID " + jobId + " failed", e);
+			}
+		} catch (final IOException e) {
+			throw new RuntimeException("Could not connect to Flink JobManager with address " + this.jobManagerHost
+					+ ":" + this.jobManagerPort, e);
+		}
+	}
+
+	// Flink specific additional methods
+
+	/**
+	 * Package internal method to get a Flink {@link JobID} from a Storm topology name.
+	 *
+	 * @param id
+	 * 		The Storm topology name.
+	 * @return Flink's internally used {@link JobID}.
+	 */
+	JobID getTopologyJobId(final String id) {
+		final Configuration configuration = GlobalConfiguration.getConfiguration();
+		if (this.timeout != null) {
+			configuration.setString(ConfigConstants.AKKA_ASK_TIMEOUT, this.timeout);
+		}
+
+		try {
+			final ActorRef jobManager = this.getJobManager();
+
+			final FiniteDuration askTimeout = this.getTimeout();
+			final Future<Object> response = Patterns.ask(jobManager, JobManagerMessages.getRequestRunningJobsStatus(),
+					new Timeout(askTimeout));
+
+			Object result;
+			try {
+				result = Await.result(response, askTimeout);
+			} catch (final Exception e) {
+				throw new RuntimeException("Could not retrieve running jobs from the JobManager", e);
+			}
+
+			if (result instanceof RunningJobsStatus) {
+				final List<JobStatusMessage> jobs = ((RunningJobsStatus) result).getStatusMessages();
+
+				for (final JobStatusMessage status : jobs) {
+					if (status.getJobName().equals(id)) {
+						return status.getJobId();
+					}
+				}
+			} else {
+				throw new RuntimeException("ReqeustRunningJobs requires a response of type "
+						+ "RunningJobs. Instead the response is of type " + result.getClass() + ".");
+			}
+		} catch (final IOException e) {
+			throw new RuntimeException("Could not connect to Flink JobManager with address " + this.jobManagerHost
+					+ ":" + this.jobManagerPort, e);
+		}
+
+		return null;
+	}
+
+	private FiniteDuration getTimeout() {
+		final Configuration configuration = GlobalConfiguration.getConfiguration();
+		if (this.timeout != null) {
+			configuration.setString(ConfigConstants.AKKA_ASK_TIMEOUT, this.timeout);
+		}
+
+		return AkkaUtils.getTimeout(configuration);
+	}
+
+	private ActorRef getJobManager() throws IOException {
+		final Configuration configuration = GlobalConfiguration.getConfiguration();
+
+		ActorSystem actorSystem;
+		try {
+			final scala.Tuple2<String, Object> systemEndpoint = new scala.Tuple2<String, Object>("", 0);
+			actorSystem = AkkaUtils.createActorSystem(configuration, new Some<scala.Tuple2<String, Object>>(
+					systemEndpoint));
+		} catch (final Exception e) {
+			throw new RuntimeException("Could not start actor system to communicate with JobManager", e);
+		}
+
+		return JobManager.getJobManagerActorRef(new InetSocketAddress(this.jobManagerHost, this.jobManagerPort),
+				actorSystem, AkkaUtils.getLookupTimeout(configuration));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java
new file mode 100644
index 0000000..868801b
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java
@@ -0,0 +1,173 @@
+/*
+ * 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.api;
+
+import backtype.storm.LocalCluster;
+import backtype.storm.generated.ClusterSummary;
+import backtype.storm.generated.KillOptions;
+import backtype.storm.generated.RebalanceOptions;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.generated.SubmitOptions;
+import backtype.storm.generated.TopologyInfo;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+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.storm.util.StormConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * {@link FlinkLocalCluster} mimics a Storm {@link LocalCluster}.
+ */
+public class FlinkLocalCluster {
+
+	/** The log used by this mini cluster */
+	private static final Logger LOG = LoggerFactory.getLogger(FlinkLocalCluster.class);
+
+	/** The flink mini cluster on which to execute the programs */
+	private final FlinkMiniCluster flink;
+
+
+	public FlinkLocalCluster() {
+		this.flink = new LocalFlinkMiniCluster(new Configuration(), true, StreamingMode.STREAMING);
+		this.flink.start();
+	}
+
+	public FlinkLocalCluster(FlinkMiniCluster flink) {
+		this.flink = Objects.requireNonNull(flink);
+	}
+
+	@SuppressWarnings("rawtypes")
+	public void submitTopology(final String topologyName, final Map conf, final FlinkTopology topology)
+			throws Exception {
+		this.submitTopologyWithOpts(topologyName, conf, topology, null);
+	}
+
+	@SuppressWarnings("rawtypes")
+	public void submitTopologyWithOpts(final String topologyName, final Map conf, final FlinkTopology topology, final SubmitOptions submitOpts) throws Exception {
+		LOG.info("Running Storm topology on FlinkLocalCluster");
+
+		if(conf != null) {
+			topology.getConfig().setGlobalJobParameters(new StormConfig(conf));
+		}
+
+		JobGraph jobGraph = topology.getStreamGraph().getJobGraph(topologyName);
+		this.flink.submitJobDetached(jobGraph);
+	}
+
+	public void killTopology(final String topologyName) {
+		this.killTopologyWithOpts(topologyName, null);
+	}
+
+	public void killTopologyWithOpts(final String name, final KillOptions options) {
+	}
+
+	public void activate(final String topologyName) {
+	}
+
+	public void deactivate(final String topologyName) {
+	}
+
+	public void rebalance(final String name, final RebalanceOptions options) {
+	}
+
+	public void shutdown() {
+		flink.stop();
+	}
+
+	public String getTopologyConf(final String id) {
+		return null;
+	}
+
+	public StormTopology getTopology(final String id) {
+		return null;
+	}
+
+	public ClusterSummary getClusterInfo() {
+		return null;
+	}
+
+	public TopologyInfo getTopologyInfo(final String id) {
+		return null;
+	}
+
+	public Map<?, ?> getState() {
+		return null;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Access to default local cluster
+	// ------------------------------------------------------------------------
+
+	// A different {@link FlinkLocalCluster} to be used for execution of ITCases
+	private static LocalClusterFactory currentFactory = new DefaultLocalClusterFactory();
+
+	/**
+	 * Returns a {@link FlinkLocalCluster} that should be used for execution. If no cluster was set by
+	 * {@link #initialize(LocalClusterFactory)} in advance, a new {@link FlinkLocalCluster} is returned.
+	 *
+	 * @return a {@link FlinkLocalCluster} to be used for execution
+	 */
+	public static FlinkLocalCluster getLocalCluster() {
+		return currentFactory.createLocalCluster();
+	}
+
+	/**
+	 * Sets a different factory for FlinkLocalClusters to be used for execution.
+	 *
+	 * @param clusterFactory
+	 * 		The LocalClusterFactory to create the local clusters for execution.
+	 */
+	public static void initialize(LocalClusterFactory clusterFactory) {
+		currentFactory = Objects.requireNonNull(clusterFactory);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Cluster factory
+	// ------------------------------------------------------------------------
+
+	/**
+	 * A factory that creates local clusters.
+	 */
+	public static interface LocalClusterFactory {
+
+		/**
+		 * Creates a local flink cluster.
+		 * @return A local flink cluster.
+		 */
+		FlinkLocalCluster createLocalCluster();
+	}
+
+	/**
+	 * A factory that instantiates a FlinkLocalCluster.
+	 */
+	public static class DefaultLocalClusterFactory implements LocalClusterFactory {
+
+		@Override
+		public FlinkLocalCluster createLocalCluster() {
+			return new FlinkLocalCluster();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java
new file mode 100644
index 0000000..88d2dfe
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java
@@ -0,0 +1,168 @@
+/*
+ * 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.api;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * {@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}.<br />
+ * <br />
+ * <strong>CAUTION: Flink does not support direct emit.</strong>
+ */
+final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer {
+
+	/** The declared output streams and schemas. */
+	final HashMap<String, Fields> outputStreams = new HashMap<String, Fields>();
+
+	@Override
+	public void declare(final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
+	}
+
+	/**
+	 * {@inheritDoc}
+	 * <p/>
+	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		if {@code direct} is {@code true}
+	 */
+	@Override
+	public void declare(final boolean direct, final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
+	}
+
+	@Override
+	public void declareStream(final String streamId, final Fields fields) {
+		this.declareStream(streamId, false, fields);
+	}
+
+	/**
+	 * {@inheritDoc}
+	 * <p/>
+	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		if {@code direct} is {@code true}
+	 */
+	@Override
+	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
+		if (direct) {
+			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
+		}
+
+		this.outputStreams.put(streamId, fields);
+	}
+
+	/**
+	 * Returns {@link TypeInformation} for the declared output schema for a specific stream.
+	 * 
+	 * @param streamId
+	 *            A stream ID.
+	 * 
+	 * @return output type information for the declared output schema of the specified stream; or {@code null} if
+	 *         {@code streamId == null}
+	 * 
+	 * @throws IllegalArgumentException
+	 *             If no output schema was declared for the specified stream or if more then 25 attributes got declared.
+	 */
+	TypeInformation<?> getOutputType(final String streamId) throws IllegalArgumentException {
+		if (streamId == null) {
+			return null;
+		}
+
+		Fields outputSchema = this.outputStreams.get(streamId);
+		if (outputSchema == null) {
+			throw new IllegalArgumentException("Stream with ID '" + streamId
+					+ "' was not declared.");
+		}
+
+		Tuple t;
+		final int numberOfAttributes = outputSchema.size();
+
+		if (numberOfAttributes == 1) {
+			return TypeExtractor.getForClass(Object.class);
+		} else if (numberOfAttributes <= 25) {
+			try {
+				t = Tuple.getTupleClass(numberOfAttributes).newInstance();
+			} catch (final InstantiationException e) {
+				throw new RuntimeException(e);
+			} catch (final IllegalAccessException e) {
+				throw new RuntimeException(e);
+			}
+		} else {
+			throw new IllegalArgumentException("Flink supports only a maximum number of 25 attributes");
+		}
+
+		// TODO: declare only key fields as DefaultComparable
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			t.setField(new DefaultComparable(), i);
+		}
+
+		return TypeExtractor.getForObject(t);
+	}
+
+	/**
+	 * {@link DefaultComparable} is a {@link Comparable} helper class that is used to get the correct {@link
+	 * TypeInformation} from {@link TypeExtractor} within {@link #getOutputType()}. If key fields are not comparable,
+	 * Flink cannot use them and will throw an exception.
+	 */
+	private static class DefaultComparable implements Comparable<DefaultComparable> {
+
+		public DefaultComparable() {
+		}
+
+		@Override
+		public int compareTo(final DefaultComparable o) {
+			return 0;
+		}
+	}
+
+	/**
+	 * Computes the indexes within the declared output schema of the specified stream, for a list of given
+	 * field-grouping attributes.
+	 * 
+	 * @param streamId
+	 *            A stream ID.
+	 * @param groupingFields
+	 *            The names of the key fields.
+	 * 
+	 * @return array of {@code int}s that contains the index within the output schema for each attribute in the given
+	 *         list
+	 */
+	int[] getGroupingFieldIndexes(final String streamId, final List<String> groupingFields) {
+		final int[] fieldIndexes = new int[groupingFields.size()];
+
+		for (int i = 0; i < fieldIndexes.length; ++i) {
+			fieldIndexes[i] = this.outputStreams.get(streamId).fieldIndex(groupingFields.get(i));
+		}
+
+		return fieldIndexes;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java
new file mode 100644
index 0000000..9b03c68
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java
@@ -0,0 +1,194 @@
+/*
+ * 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.api;
+
+import backtype.storm.Config;
+import backtype.storm.StormSubmitter;
+import backtype.storm.generated.AlreadyAliveException;
+import backtype.storm.generated.InvalidTopologyException;
+import backtype.storm.generated.SubmitOptions;
+import backtype.storm.utils.Utils;
+
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.client.program.ContextEnvironment;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.json.simple.JSONValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Map;
+
+/**
+ * {@link FlinkSubmitter} mimics a {@link StormSubmitter} to submit Storm topologies to a Flink cluster.
+ */
+public class FlinkSubmitter {
+	public final static Logger logger = LoggerFactory.getLogger(FlinkSubmitter.class);
+
+	/**
+	 * Submits a topology to run on the cluster. A topology runs forever or until explicitly killed.
+	 *
+	 * @param name
+	 * 		the name of the storm.
+	 * @param stormConf
+	 * 		the topology-specific configuration. See {@link Config}.
+	 * @param topology
+	 * 		the processing to execute.
+	 * @param opts
+	 * 		to manipulate the starting of the topology.
+	 * @throws AlreadyAliveException
+	 * 		if a topology with this name is already running
+	 * @throws InvalidTopologyException
+	 * 		if an invalid topology was submitted
+	 */
+	public static void submitTopology(final String name, final Map<?, ?> stormConf, final FlinkTopology topology,
+			final SubmitOptions opts)
+					throws AlreadyAliveException, InvalidTopologyException {
+		submitTopology(name, stormConf, topology);
+	}
+
+	/**
+	 * Submits a topology to run on the cluster. A topology runs forever or until explicitly killed. The given {@link
+	 * FlinkProgressListener} is ignored because progress bars are not supported by Flink.
+	 *
+	 * @param name
+	 * 		the name of the storm.
+	 * @param stormConf
+	 * 		the topology-specific configuration. See {@link Config}.
+	 * @param topology
+	 * 		the processing to execute.
+	 * @throws AlreadyAliveException
+	 * 		if a topology with this name is already running
+	 * @throws InvalidTopologyException
+	 * 		if an invalid topology was submitted
+	 */
+	@SuppressWarnings({"rawtypes", "unchecked"})
+	public static void submitTopology(final String name, final Map stormConf, final FlinkTopology topology)
+			throws AlreadyAliveException, InvalidTopologyException {
+		if (!Utils.isValidConf(stormConf)) {
+			throw new IllegalArgumentException("Storm conf is not valid. Must be json-serializable");
+		}
+
+		final Configuration flinkConfig = GlobalConfiguration.getConfiguration();
+		if (!stormConf.containsKey(Config.NIMBUS_HOST)) {
+			stormConf.put(Config.NIMBUS_HOST,
+					flinkConfig.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost"));
+		}
+		if (!stormConf.containsKey(Config.NIMBUS_THRIFT_PORT)) {
+			stormConf.put(Config.NIMBUS_THRIFT_PORT,
+					new Integer(flinkConfig.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+							6123)));
+		}
+
+		final String serConf = JSONValue.toJSONString(stormConf);
+
+		final FlinkClient client = FlinkClient.getConfiguredClient(stormConf);
+		try {
+			if (client.getTopologyJobId(name) != null) {
+				throw new RuntimeException("Topology with name `" + name + "` already exists on cluster");
+			}
+			String localJar = System.getProperty("storm.jar");
+			if (localJar == null) {
+				try {
+					for (final File file : ((ContextEnvironment) ExecutionEnvironment.getExecutionEnvironment())
+							.getJars()) {
+						// TODO verify that there is only one jar
+						localJar = file.getAbsolutePath();
+					}
+				} catch (final ClassCastException e) {
+					// ignore
+				}
+			}
+
+			logger.info("Submitting topology " + name + " in distributed mode with conf " + serConf);
+			client.submitTopologyWithOpts(name, localJar, topology);
+		} catch (final InvalidTopologyException e) {
+			logger.warn("Topology submission exception: " + e.get_msg());
+			throw e;
+		} catch (final AlreadyAliveException e) {
+			logger.warn("Topology already alive exception", e);
+			throw e;
+		}
+
+		logger.info("Finished submitting topology: " + name);
+	}
+
+	/**
+	 * Same as {@link #submitTopology(String, Map, FlinkTopology, SubmitOptions)}. Progress bars are not supported by
+	 * Flink.
+	 *
+	 * @param name
+	 * 		the name of the storm.
+	 * @param stormConf
+	 * 		the topology-specific configuration. See {@link Config}.
+	 * @param topology
+	 * 		the processing to execute.
+	 * @throws AlreadyAliveException
+	 * 		if a topology with this name is already running
+	 * @throws InvalidTopologyException
+	 * 		if an invalid topology was submitted
+	 */
+	public static void submitTopologyWithProgressBar(final String name, final Map<?, ?> stormConf,
+			final FlinkTopology topology)
+					throws AlreadyAliveException, InvalidTopologyException {
+		submitTopology(name, stormConf, topology);
+	}
+
+	/**
+	 * In Flink, jar files are submitted directly when a program is started. Thus, this method does nothing. The
+	 * returned value is parameter localJar, because this give the best integration of Storm behavior within a Flink
+	 * environment.
+	 *
+	 * @param conf
+	 * 		the topology-specific configuration. See {@link Config}.
+	 * @param localJar
+	 * 		file path of the jar file to submit
+	 * @return the value of parameter localJar
+	 */
+	@SuppressWarnings("rawtypes")
+	public static String submitJar(final Map conf, final String localJar) {
+		return submitJar(localJar);
+	}
+
+	/**
+	 * In Flink, jar files are submitted directly when a program is started. Thus, this method does nothing. The
+	 * returned value is parameter localJar, because this give the best integration of Storm behavior within a Flink
+	 * environment.
+	 *
+	 * @param localJar
+	 * 		file path of the jar file to submit
+	 * @return the value of parameter localJar
+	 */
+	public static String submitJar(final String localJar) {
+		if (localJar == null) {
+			throw new RuntimeException(
+					"Must submit topologies using the 'storm' client script so that StormSubmitter knows which jar " +
+					"to upload");
+		}
+
+		return localJar;
+	}
+
+	/**
+	 * Dummy interface use to track progress of file upload. Does not do anything. Kept for compatibility.
+	 */
+	public interface FlinkProgressListener {
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java
new file mode 100644
index 0000000..531d6df
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java
@@ -0,0 +1,89 @@
+/*
+ * 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.api;
+
+import backtype.storm.generated.StormTopology;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+/**
+ * {@link FlinkTopology} mimics a {@link StormTopology} and is implemented in terms of a {@link
+ * StreamExecutionEnvironment} . In contrast to a regular {@link StreamExecutionEnvironment}, a {@link FlinkTopology}
+ * cannot be executed directly, but must be handed over to a {@link FlinkLocalCluster}, {@link FlinkSubmitter}, or
+ * {@link FlinkClient}.
+ */
+public class FlinkTopology extends StreamExecutionEnvironment {
+
+	/** The number of declared tasks for the whole program (ie, sum over all dops) */
+	private int numberOfTasks = 0;
+
+	public FlinkTopology() {
+		// Set default parallelism to 1, to mirror Storm default behavior
+		super.setParallelism(1);
+	}
+
+	/**
+	 * Is not supported. In order to execute use {@link FlinkLocalCluster}, {@link FlinkSubmitter}, or {@link
+	 * FlinkClient}.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public JobExecutionResult execute() throws Exception {
+		throw new UnsupportedOperationException(
+				"A FlinkTopology cannot be executed directly. Use FlinkLocalCluster, FlinkSubmitter, or FlinkClient " +
+				"instead.");
+	}
+
+	/**
+	 * Is not supported. In order to execute use {@link FlinkLocalCluster}, {@link FlinkSubmitter} or {@link
+	 * FlinkClient}.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public JobExecutionResult execute(final String jobName) throws Exception {
+		throw new UnsupportedOperationException(
+				"A FlinkTopology cannot be executed directly. Use FlinkLocalCluster, FlinkSubmitter, or FlinkClient " +
+				"instead.");
+	}
+
+	/**
+	 * Increased the number of declared tasks of this program by the given value.
+	 *
+	 * @param dop
+	 * 		The dop of a new operator that increases the number of overall tasks.
+	 */
+	public void increaseNumberOfTasks(final int dop) {
+		assert (dop > 0);
+		this.numberOfTasks += dop;
+	}
+
+	/**
+	 * Return the number or required tasks to execute this program.
+	 *
+	 * @return the number or required tasks to execute this program
+	 */
+	public int getNumberOfTasks() {
+		return this.numberOfTasks;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
new file mode 100644
index 0000000..99de0e2
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
@@ -0,0 +1,397 @@
+/*
+/*
+ * 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.api;
+
+import backtype.storm.generated.ComponentCommon;
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.generated.Grouping;
+import backtype.storm.generated.StormTopology;
+import backtype.storm.topology.BasicBoltExecutor;
+import backtype.storm.topology.BoltDeclarer;
+import backtype.storm.topology.IBasicBolt;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.IRichStateSpout;
+import backtype.storm.topology.SpoutDeclarer;
+import backtype.storm.topology.TopologyBuilder;
+import backtype.storm.tuple.Fields;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+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;
+import org.apache.flink.streaming.api.datastream.SplitStream;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+
+/**
+ * {@link FlinkTopologyBuilder} mimics a {@link TopologyBuilder}, but builds a Flink program instead of a Storm
+ * topology. Most methods (except {@link #createTopology()} are copied from the original {@link TopologyBuilder}
+ * implementation to ensure equal behavior.<br />
+ * <br />
+ * <strong>CAUTION: {@link IRichStateSpout StateSpout}s are currently not supported.</strong>
+ */
+public class FlinkTopologyBuilder {
+
+	/** A Storm {@link TopologyBuilder} to build a real Storm topology */
+	private final TopologyBuilder stormBuilder = new TopologyBuilder();
+	/** All user spouts by their ID */
+	private final HashMap<String, IRichSpout> spouts = new HashMap<String, IRichSpout>();
+	/** All user bolts by their ID */
+	private final HashMap<String, IRichBolt> bolts = new HashMap<String, IRichBolt>();
+	/** All declared streams and output schemas by operator ID */
+	private final HashMap<String, HashMap<String, Fields>> outputStreams = new HashMap<String, HashMap<String, Fields>>();
+	/** All spouts&bolts declarers by their ID */
+	private final HashMap<String, FlinkOutputFieldsDeclarer> declarers = new HashMap<String, FlinkOutputFieldsDeclarer>();
+	// 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() {
+		this.stormTopology = this.stormBuilder.createTopology();
+
+		final FlinkTopology env = new FlinkTopology();
+		env.setParallelism(1);
+
+		final HashMap<String, HashMap<String, DataStream>> availableInputs = new HashMap<String, HashMap<String, DataStream>>();
+
+		for (final Entry<String, IRichSpout> spout : this.spouts.entrySet()) {
+			final String spoutId = spout.getKey();
+			final IRichSpout userSpout = spout.getValue();
+
+			final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+			userSpout.declareOutputFields(declarer);
+			final HashMap<String,Fields> sourceStreams = declarer.outputStreams;
+			this.outputStreams.put(spoutId, sourceStreams);
+			declarers.put(spoutId, declarer);
+
+			final SpoutWrapper spoutWrapper = new SpoutWrapper(userSpout);
+			spoutWrapper.setStormTopology(stormTopology);
+
+			DataStreamSource source;
+			final HashMap<String, DataStream> outputStreams = new HashMap<String, DataStream>();
+			if (sourceStreams.size() == 1) {
+				final String outputStreamId = (String) sourceStreams.keySet().toArray()[0];
+				source = env.addSource(spoutWrapper, spoutId,
+						declarer.getOutputType(outputStreamId));
+				outputStreams.put(outputStreamId, source);
+			} else {
+				source = env.addSource(spoutWrapper, spoutId,
+						TypeExtractor.getForClass(SplitStreamType.class));
+				SplitStream splitSource = source.split(new StormStreamSelector());
+
+				for (String streamId : sourceStreams.keySet()) {
+					outputStreams.put(streamId, splitSource.select(streamId));
+				}
+			}
+			availableInputs.put(spoutId, outputStreams);
+
+			int dop = 1;
+			final ComponentCommon common = stormTopology.get_spouts().get(spoutId).get_common();
+			if (common.is_set_parallelism_hint()) {
+				dop = common.get_parallelism_hint();
+				source.setParallelism(dop);
+			} else {
+				common.set_parallelism_hint(1);
+			}
+			env.increaseNumberOfTasks(dop);
+		}
+
+		final HashMap<String, IRichBolt> unprocessedBolts = new HashMap<String, IRichBolt>();
+		unprocessedBolts.putAll(this.bolts);
+
+		final HashMap<String, Set<Entry<GlobalStreamId, Grouping>>> unprocessdInputsPerBolt =
+				new HashMap<String, Set<Entry<GlobalStreamId, Grouping>>>();
+
+		/* Because we do not know the order in which an iterator steps over a set, we might process a consumer before
+		 * its producer
+		 * ->thus, we might need to repeat multiple times
+		 */
+		boolean makeProgress = true;
+		while (unprocessedBolts.size() > 0) {
+			if (!makeProgress) {
+				throw new RuntimeException(
+						"Unable to build Topology. Could not connect the following bolts: "
+								+ unprocessedBolts.keySet());
+			}
+			makeProgress = false;
+
+			final Iterator<Entry<String, IRichBolt>> boltsIterator = unprocessedBolts.entrySet().iterator();
+			while (boltsIterator.hasNext()) {
+
+				final Entry<String, IRichBolt> bolt = boltsIterator.next();
+				final String boltId = bolt.getKey();
+				final IRichBolt userBolt = bolt.getValue();
+
+				final ComponentCommon common = stormTopology.get_bolts().get(boltId).get_common();
+
+				Set<Entry<GlobalStreamId, Grouping>> unprocessedInputs = unprocessdInputsPerBolt.get(boltId);
+				if (unprocessedInputs == null) {
+					unprocessedInputs = new HashSet<Entry<GlobalStreamId, Grouping>>();
+					unprocessedInputs.addAll(common.get_inputs().entrySet());
+					unprocessdInputsPerBolt.put(boltId, unprocessedInputs);
+				}
+
+				// connect each available producer to the current bolt
+				final Iterator<Entry<GlobalStreamId, Grouping>> inputStreamsIterator = unprocessedInputs.iterator();
+				while (inputStreamsIterator.hasNext()) {
+
+					final Entry<GlobalStreamId, Grouping> stormInputStream = inputStreamsIterator.next();
+					final String producerId = stormInputStream.getKey().get_componentId();
+					final String inputStreamId = stormInputStream.getKey().get_streamId();
+
+					final HashMap<String, DataStream> producer = availableInputs.get(producerId);
+					if (producer != null) {
+						makeProgress = true;
+
+						DataStream inputStream = producer.get(inputStreamId);
+						if (inputStream != null) {
+							final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+							userBolt.declareOutputFields(declarer);
+							final HashMap<String, Fields> boltOutputStreams = declarer.outputStreams;
+							this.outputStreams.put(boltId, boltOutputStreams);
+							this.declarers.put(boltId, declarer);
+
+							// if producer was processed already
+							final Grouping grouping = stormInputStream.getValue();
+							if (grouping.is_set_shuffle()) {
+								// Storm uses a round-robin shuffle strategy
+								inputStream = inputStream.rebalance();
+							} else if (grouping.is_set_fields()) {
+								// global grouping is emulated in Storm via an empty fields grouping list
+								final List<String> fields = grouping.get_fields();
+								if (fields.size() > 0) {
+									FlinkOutputFieldsDeclarer prodDeclarer = this.declarers.get(producerId);
+									if (producer.size() == 1) {
+										inputStream = inputStream.keyBy(prodDeclarer
+												.getGroupingFieldIndexes(inputStreamId,
+														grouping.get_fields()));
+									} else {
+										inputStream = inputStream
+												.keyBy(new SplitStreamTypeKeySelector(
+														prodDeclarer.getGroupingFieldIndexes(
+																inputStreamId,
+																grouping.get_fields())));
+									}
+								} else {
+									inputStream = inputStream.global();
+								}
+							} else if (grouping.is_set_all()) {
+								inputStream = inputStream.broadcast();
+							} else if (!grouping.is_set_local_or_shuffle()) {
+								throw new UnsupportedOperationException(
+										"Flink only supports (local-or-)shuffle, fields, all, and global grouping");
+							}
+
+							final SingleOutputStreamOperator outputStream;
+							final BoltWrapper boltWrapper;
+							if (boltOutputStreams.size() < 2) { // single output stream or sink
+								String outputStreamId = null;
+								if (boltOutputStreams.size() == 1) {
+									outputStreamId = (String) boltOutputStreams.keySet().toArray()[0];
+								}
+								final TypeInformation<?> outType = declarer
+										.getOutputType(outputStreamId);
+
+								boltWrapper = new BoltWrapper(userBolt, this.outputStreams
+										.get(producerId).get(inputStreamId));
+								outputStream = inputStream.transform(boltId, outType, boltWrapper);
+
+								if (outType != null) {
+									// only for non-sink nodes
+									final HashMap<String, DataStream> op = new HashMap<String, DataStream>();
+									op.put(outputStreamId, outputStream);
+									availableInputs.put(boltId, op);
+								}
+							} else {
+								final TypeInformation<?> outType = TypeExtractor
+										.getForClass(SplitStreamType.class);
+
+								boltWrapper = new BoltWrapper(userBolt, this.outputStreams.get(producerId).get(inputStreamId));
+								outputStream = inputStream.transform(boltId, outType, boltWrapper);
+
+								final SplitStream splitStreams = outputStream
+										.split(new StormStreamSelector());
+
+								final HashMap<String, DataStream> op = new HashMap<String, DataStream>();
+								for (String outputStreamId : boltOutputStreams.keySet()) {
+									op.put(outputStreamId, splitStreams.select(outputStreamId));
+								}
+								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);
+
+							inputStreamsIterator.remove();
+						} else {
+							throw new RuntimeException("Cannot connect '" + boltId + "' to '"
+									+ producerId + "'. Stream '" + inputStreamId + "' not found.");
+						}
+					}
+				}
+
+				if (unprocessedInputs.size() == 0) {
+					// all inputs are connected; processing bolt completed
+					boltsIterator.remove();
+				}
+			}
+		}
+		return env;
+	}
+
+	/**
+	 * Define a new bolt in this topology with parallelism of just one thread.
+	 *
+	 * @param id
+	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
+	 * 		outputs.
+	 * @param bolt
+	 * 		the bolt
+	 * @return use the returned object to declare the inputs to this component
+	 */
+	public BoltDeclarer setBolt(final String id, final IRichBolt bolt) {
+		return this.setBolt(id, bolt, null);
+	}
+
+	/**
+	 * Define a new bolt in this topology with the specified amount of parallelism.
+	 *
+	 * @param id
+	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
+	 * 		outputs.
+	 * @param bolt
+	 * 		the bolt
+	 * @param parallelism_hint
+	 * 		the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a
+	 * 		process somewhere around the cluster.
+	 * @return use the returned object to declare the inputs to this component
+	 */
+	public BoltDeclarer setBolt(final String id, final IRichBolt bolt, final Number parallelism_hint) {
+		final BoltDeclarer declarer = this.stormBuilder.setBolt(id, bolt, parallelism_hint);
+		this.bolts.put(id, bolt);
+		return declarer;
+	}
+
+	/**
+	 * Define a new bolt in this topology. This defines a basic bolt, which is a simpler to use but more restricted
+	 * kind
+	 * of bolt. Basic bolts are intended for non-aggregation processing and automate the anchoring/acking process to
+	 * achieve proper reliability in the topology.
+	 *
+	 * @param id
+	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
+	 * 		outputs.
+	 * @param bolt
+	 * 		the basic bolt
+	 * @return use the returned object to declare the inputs to this component
+	 */
+	public BoltDeclarer setBolt(final String id, final IBasicBolt bolt) {
+		return this.setBolt(id, bolt, null);
+	}
+
+	/**
+	 * Define a new bolt in this topology. This defines a basic bolt, which is a simpler to use but more restricted
+	 * kind
+	 * of bolt. Basic bolts are intended for non-aggregation processing and automate the anchoring/acking process to
+	 * achieve proper reliability in the topology.
+	 *
+	 * @param id
+	 * 		the id of this component. This id is referenced by other components that want to consume this bolt's
+	 * 		outputs.
+	 * @param bolt
+	 * 		the basic bolt
+	 * @param parallelism_hint
+	 * 		the number of tasks that should be assigned to execute this bolt. Each task will run on a thread in a
+	 * 		process somwehere around the cluster.
+	 * @return use the returned object to declare the inputs to this component
+	 */
+	public BoltDeclarer setBolt(final String id, final IBasicBolt bolt, final Number parallelism_hint) {
+		return this.setBolt(id, new BasicBoltExecutor(bolt), parallelism_hint);
+	}
+
+	/**
+	 * Define a new spout in this topology.
+	 *
+	 * @param id
+	 * 		the id of this component. This id is referenced by other components that want to consume this spout's
+	 * 		outputs.
+	 * @param spout
+	 * 		the spout
+	 */
+	public SpoutDeclarer setSpout(final String id, final IRichSpout spout) {
+		return this.setSpout(id, spout, null);
+	}
+
+	/**
+	 * Define a new spout in this topology with the specified parallelism. If the spout declares itself as
+	 * non-distributed, the parallelism_hint will be ignored and only one task will be allocated to this component.
+	 *
+	 * @param id
+	 * 		the id of this component. This id is referenced by other components that want to consume this spout's
+	 * 		outputs.
+	 * @param parallelism_hint
+	 * 		the number of tasks that should be assigned to execute this spout. Each task will run on a thread in a
+	 * 		process somwehere around the cluster.
+	 * @param spout
+	 * 		the spout
+	 */
+	public SpoutDeclarer setSpout(final String id, final IRichSpout spout, final Number parallelism_hint) {
+		final SpoutDeclarer declarer = this.stormBuilder.setSpout(id, spout, parallelism_hint);
+		this.spouts.put(id, spout);
+		return declarer;
+	}
+
+	// TODO add StateSpout support (Storm 0.9.4 does not yet support StateSpouts itself)
+	/* not implemented by Storm 0.9.4
+	 * public void setStateSpout(final String id, final IRichStateSpout stateSpout) {
+	 * this.stormBuilder.setStateSpout(id, stateSpout);
+	 * }
+	 * public void setStateSpout(final String id, final IRichStateSpout stateSpout, final Number parallelism_hint) {
+	 * this.stormBuilder.setStateSpout(id, stateSpout, parallelism_hint);
+	 * }
+	 */
+
+	// for internal testing purpose only
+	StormTopology getStormTopology() {
+		return this.stormTopology;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java
new file mode 100644
index 0000000..99c2583
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java
@@ -0,0 +1,36 @@
+/*
+ * 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.util;
+
+import backtype.storm.topology.IRichSpout;
+
+/**
+ * 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 FiniteSpout extends IRichSpout {
+
+	/**
+	 * When returns true, the spout has reached the end of the stream.
+	 *
+	 * @return true, if the spout's stream reached its end, false otherwise
+	 */
+	public boolean reachedEnd();
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamMapper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamMapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamMapper.java
new file mode 100644
index 0000000..1fb5e02
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamMapper.java
@@ -0,0 +1,39 @@
+/*
+ * 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.util;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SplitStream;
+
+/**
+ * Strips {@link SplitStreamType}{@code <T>} away, ie, extracts the wrapped record of type {@code T}. Can be used to get
+ * a "clean" stream from a Spout/Bolt that declared multiple output streams (after the streams got separated using
+ * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector) .split(...)} and
+ * {@link SplitStream#select(String...) .select(...)}).
+ * 
+ * @param <T>
+ */
+public class SplitStreamMapper<T> implements MapFunction<SplitStreamType<T>, T> {
+	private static final long serialVersionUID = 3550359150160908564L;
+
+	@Override
+	public T map(SplitStreamType<T> value) throws Exception {
+		return value.value;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java
new file mode 100644
index 0000000..a4b5f8e
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamType.java
@@ -0,0 +1,52 @@
+/*
+ * 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.util;
+
+import org.apache.flink.streaming.api.datastream.DataStream;
+
+/**
+ * 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 StormStreamSelector}.
+ */
+public class SplitStreamType<T> {
+
+	/** The stream ID this tuple belongs to. */
+	public String streamId;
+	/** The actual data value. */
+	public T value;
+
+	@Override
+	public String toString() {
+		return "<sid:" + this.streamId + ",v:" + this.value + ">";
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+		SplitStreamType<?> other = (SplitStreamType<?>) o;
+
+		return this.streamId.equals(other.streamId) && this.value.equals(other.value);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java
new file mode 100644
index 0000000..44c693c
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/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.storm.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 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.
+ */
+public class SplitStreamTypeKeySelector implements KeySelector<SplitStreamType<Tuple>, Tuple> {
+	private static final long serialVersionUID = 4672434660037669254L;
+
+	private final ArrayKeySelector<Tuple> selector;
+
+	public SplitStreamTypeKeySelector(int... fields) {
+		this.selector = new KeySelectorUtil.ArrayKeySelector<Tuple>(fields);
+	}
+
+	@Override
+	public Tuple getKey(SplitStreamType<Tuple> value) throws Exception {
+		return selector.getKey(value.value);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java
new file mode 100644
index 0000000..6550990
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java
@@ -0,0 +1,123 @@
+/*
+ * 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.util;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters;
+
+import backtype.storm.Config;
+
+/**
+ * {@link StormConfig} is used to provide a user-defined Storm configuration (ie, a raw {@link Map} or {@link Config}
+ * object) for embedded Spouts and Bolts.
+ */
+@SuppressWarnings("rawtypes")
+public final class StormConfig extends GlobalJobParameters implements Map {
+	private static final long serialVersionUID = 8019519109673698490L;
+
+	/** Contains the actual configuration that is provided to Spouts and Bolts. */
+	private final Map config = new HashMap();
+
+	/**
+	 * Creates an empty configuration.
+	 */
+	public StormConfig() {
+	}
+
+	/**
+	 * Creates an configuration with initial values provided by the given {@code Map}.
+	 * 
+	 * @param config
+	 *            Initial values for this configuration.
+	 */
+	@SuppressWarnings("unchecked")
+	public StormConfig(Map config) {
+		this.config.putAll(config);
+	}
+
+
+	@Override
+	public int size() {
+		return this.config.size();
+	}
+
+	@Override
+	public boolean isEmpty() {
+		return this.config.isEmpty();
+	}
+
+	@Override
+	public boolean containsKey(Object key) {
+		return this.config.containsKey(key);
+	}
+
+	@Override
+	public boolean containsValue(Object value) {
+		return this.config.containsValue(value);
+	}
+
+	@Override
+	public Object get(Object key) {
+		return this.config.get(key);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public Object put(Object key, Object value) {
+		return this.config.put(key, value);
+	}
+
+	@Override
+	public Object remove(Object key) {
+		return this.config.remove(key);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void putAll(Map m) {
+		this.config.putAll(m);
+	}
+
+	@Override
+	public void clear() {
+		this.config.clear();
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public Set<Object> keySet() {
+		return this.config.keySet();
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public Collection<Object> values() {
+		return this.config.values();
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public Set<java.util.Map.Entry<Object, Object>> entrySet() {
+		return this.config.entrySet();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java
new file mode 100644
index 0000000..d9f4178
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormStreamSelector.java
@@ -0,0 +1,48 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+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 StormStreamSelector<T> implements OutputSelector<SplitStreamType<T>> {
+	private static final long serialVersionUID = 2553423379715401023L;
+
+	/** internal cache to avoid short living ArrayList objects. */
+	private final HashMap<String, List<String>> streams = new HashMap<String, List<String>>();
+
+	@Override
+	public Iterable<String> select(SplitStreamType<T> value) {
+		String sid = value.streamId;
+		List<String> streamId = this.streams.get(sid);
+		if (streamId == null) {
+			streamId = new ArrayList<String>(1);
+			streamId.add(sid);
+			this.streams.put(sid, streamId);
+		}
+		return streamId;
+	}
+
+}
\ No newline at end of file


[02/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/AbstractTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/AbstractTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/AbstractTest.java
new file mode 100644
index 0000000..f51aba4
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/AbstractTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.util;
+
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Random;
+
+public abstract class AbstractTest {
+	private static final Logger LOG = LoggerFactory.getLogger(AbstractTest.class);
+
+	protected long seed;
+	protected Random r;
+
+	@Before
+	public void prepare() {
+		this.seed = System.currentTimeMillis();
+		this.r = new Random(this.seed);
+		LOG.info("Test seed: {}", new Long(this.seed));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java
new file mode 100644
index 0000000..1b320e5
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java
@@ -0,0 +1,77 @@
+/*
+ * 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.util;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+
+import java.util.Map;
+
+public class FiniteTestSpout implements IRichSpout {
+	private static final long serialVersionUID = 7992419478267824279L;
+
+	private int numberOfOutputTuples;
+	private SpoutOutputCollector collector;
+
+	public FiniteTestSpout(final int numberOfOutputTuples) {
+		this.numberOfOutputTuples = numberOfOutputTuples;
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void close() {/* nothing to do */}
+
+	@Override
+	public void activate() {/* nothing to do */}
+
+	@Override
+	public void deactivate() {/* nothing to do */}
+
+	@Override
+	public void nextTuple() {
+		if (--this.numberOfOutputTuples >= 0) {
+			this.collector.emit(new Values(new Integer(this.numberOfOutputTuples)));
+		}
+	}
+
+	@Override
+	public void ack(final Object msgId) {/* nothing to do */}
+
+	@Override
+	public void fail(final Object msgId) {/* nothing to do */}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields("dummy"));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java
new file mode 100644
index 0000000..17de427
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/StormStreamSelectorTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.util;
+
+import java.util.Iterator;
+
+import org.apache.flink.storm.util.SplitStreamType;
+import org.apache.flink.storm.util.StormStreamSelector;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class StormStreamSelectorTest {
+
+	@Test
+	public void testSelector() {
+		StormStreamSelector<Object> selector = new StormStreamSelector<Object>();
+		SplitStreamType<Object> tuple = new SplitStreamType<Object>();
+		Iterator<String> result;
+
+		tuple.streamId = "stream1";
+		result = selector.select(tuple).iterator();
+		Assert.assertEquals("stream1", result.next());
+		Assert.assertFalse(result.hasNext());
+
+		tuple.streamId = "stream2";
+		result = selector.select(tuple).iterator();
+		Assert.assertEquals("stream2", result.next());
+		Assert.assertFalse(result.hasNext());
+
+		tuple.streamId = "stream1";
+		result = selector.select(tuple).iterator();
+		Assert.assertEquals("stream1", result.next());
+		Assert.assertFalse(result.hasNext());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java
new file mode 100644
index 0000000..b7458df
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java
@@ -0,0 +1,74 @@
+/*
+ * 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.util;
+
+import java.util.Map;
+
+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.tuple.Values;
+
+public class TestDummyBolt implements IRichBolt {
+	private static final long serialVersionUID = 6893611247443121322L;
+
+	public final static String shuffleStreamId = "shuffleStream";
+	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;
+	}
+
+	@Override
+	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() {}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {
+		declarer.declareStream(shuffleStreamId, new Fields("data"));
+		declarer.declareStream(groupingStreamId, new Fields("id", "data"));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java
new file mode 100644
index 0000000..ed9ffff
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java
@@ -0,0 +1,82 @@
+/*
+ * 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.util;
+
+import java.util.Map;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+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 {
+	private static final long serialVersionUID = -5190945609124603118L;
+
+	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;
+	}
+
+	@Override
+	public void close() {}
+
+	@Override
+	public void activate() {}
+
+	@Override
+	public void deactivate() {}
+
+	@Override
+	public void nextTuple() {
+		if (this.emit) {
+			this.collector.emit(new Values(this.context));
+			this.emit = false;
+		}
+	}
+
+	@Override
+	public void ack(Object msgId) {}
+
+	@Override
+	public void fail(Object msgId) {}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {
+		declarer.declareStream(Utils.DEFAULT_STREAM_ID, new Fields("data"));
+		declarer.declareStream(spoutStreamId, new Fields("id", "data"));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java
new file mode 100644
index 0000000..59939fd
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java
@@ -0,0 +1,60 @@
+/*
+ * 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.util;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+
+public class TestSink implements IRichBolt {
+	private static final long serialVersionUID = 4314871456719370877L;
+
+	public final static List<TopologyContext> result = new LinkedList<TopologyContext>();
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+		result.add(context);
+	}
+
+	@Override
+	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() {}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java
new file mode 100644
index 0000000..3d7d26b
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.storm.wrappers;
+
+import backtype.storm.tuple.Values;
+
+import org.apache.flink.api.java.tuple.Tuple;
+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;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class BoltCollectorTest extends AbstractTest {
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	@Test
+	public void testBoltStormCollector() throws InstantiationException, IllegalAccessException {
+		for (int numberOfAttributes = -1; numberOfAttributes < 26; ++numberOfAttributes) {
+			final Output flinkCollector = mock(Output.class);
+			Tuple flinkTuple = null;
+			final Values tuple = new Values();
+
+			BoltCollector<?> collector;
+
+			final String streamId = "streamId";
+			HashMap<String, Integer> attributes = new HashMap<String, Integer>();
+			attributes.put(streamId, numberOfAttributes);
+
+			if (numberOfAttributes == -1) {
+				collector = new BoltCollector(attributes, flinkCollector);
+				tuple.add(new Integer(this.r.nextInt()));
+
+			} else {
+				collector = new BoltCollector(attributes, flinkCollector);
+				flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
+
+				for (int i = 0; i < numberOfAttributes; ++i) {
+					tuple.add(new Integer(this.r.nextInt()));
+					flinkTuple.setField(tuple.get(i), i);
+				}
+			}
+
+			final Collection anchors = mock(Collection.class);
+			final List<Integer> taskIds;
+			taskIds = collector.emit(streamId, anchors, tuple);
+
+			Assert.assertNull(taskIds);
+
+			if (numberOfAttributes == -1) {
+				verify(flinkCollector).collect(tuple.get(0));
+			} else {
+				verify(flinkCollector).collect(flinkTuple);
+			}
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	@Test(expected = UnsupportedOperationException.class)
+	public void testEmitDirect() {
+		new BoltCollector<Object>(mock(HashMap.class), mock(Output.class)).emitDirect(0, null,
+				null, null);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java
new file mode 100644
index 0000000..e33fdb9
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java
@@ -0,0 +1,355 @@
+/*
+ * 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.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.Values;
+import backtype.storm.utils.Utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+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.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;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+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;
+import static org.mockito.Matchers.isNull;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.same;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({StreamRecordSerializer.class, WrapperSetupHelper.class})
+public class BoltWrapperTest extends AbstractTest {
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testWrapperRawType() throws Exception {
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+		declarer.declare(new Fields("dummy1", "dummy2"));
+		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		new BoltWrapper<Object, Object>(mock(IRichBolt.class),
+				new String[] { Utils.DEFAULT_STREAM_ID });
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testWrapperToManyAttributes1() throws Exception {
+		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(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		new BoltWrapper<Object, Object>(mock(IRichBolt.class));
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testWrapperToManyAttributes2() throws Exception {
+		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(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		new BoltWrapper<Object, Object>(mock(IRichBolt.class), new String[] {});
+	}
+
+	@Test
+	public void testWrapper() throws Exception {
+		for (int i = -1; i < 26; ++i) {
+			this.testWrapper(i);
+		}
+	}
+
+	@SuppressWarnings({"rawtypes", "unchecked"})
+	private void testWrapper(final int numberOfAttributes) throws Exception {
+		assert ((-1 <= numberOfAttributes) && (numberOfAttributes <= 25));
+		Tuple flinkTuple = null;
+		String rawTuple = null;
+
+		if (numberOfAttributes == -1) {
+			rawTuple = "test";
+		} else {
+			flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
+		}
+
+		final String[] schema;
+		if (numberOfAttributes == -1) {
+			schema = new String[1];
+		} else {
+			schema = new String[numberOfAttributes];
+		}
+		for (int i = 0; i < schema.length; ++i) {
+			schema[i] = "a" + i;
+		}
+
+		final StreamRecord record = mock(StreamRecord.class);
+		if (numberOfAttributes == -1) {
+			when(record.getValue()).thenReturn(rawTuple);
+		} else {
+			when(record.getValue()).thenReturn(flinkTuple);
+		}
+
+		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 SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+		declarer.declare(new Fields(schema));
+		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		final BoltWrapper wrapper = new BoltWrapper(bolt, (Fields) null);
+		wrapper.setup(mock(Output.class), taskContext);
+		wrapper.open(null);
+
+		wrapper.processElement(record);
+		if (numberOfAttributes == -1) {
+			verify(bolt).execute(eq(new StormTuple<String>(rawTuple, null)));
+		} else {
+			verify(bolt).execute(eq(new StormTuple<Tuple>(flinkTuple, null)));
+		}
+	}
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	@Test
+	public void testMultipleOutputStreams() throws Exception {
+		final boolean rawOutType1 = super.r.nextBoolean();
+		final boolean rawOutType2 = super.r.nextBoolean();
+
+		final StreamRecord record = mock(StreamRecord.class);
+		when(record.getValue()).thenReturn(2).thenReturn(3);
+
+		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);
+
+		final TestBolt bolt = new TestBolt();
+		final HashSet<String> raw = new HashSet<String>();
+		if (rawOutType1) {
+			raw.add("stream1");
+		}
+		if (rawOutType2) {
+			raw.add("stream2");
+		}
+
+		final BoltWrapper wrapper = new BoltWrapper(bolt, (Fields) null, raw);
+		wrapper.setup(output, taskContext);
+		wrapper.open(null);
+
+		final SplitStreamType splitRecord = new SplitStreamType<Integer>();
+		if (rawOutType1) {
+			splitRecord.streamId = "stream1";
+			splitRecord.value = 2;
+		} else {
+			splitRecord.streamId = "stream1";
+			splitRecord.value = new Tuple1<Integer>(2);
+		}
+		wrapper.processElement(record);
+		verify(output).collect(new StreamRecord<SplitStreamType>(splitRecord, 0));
+
+		if (rawOutType2) {
+			splitRecord.streamId = "stream2";
+			splitRecord.value = 3;
+		} else {
+			splitRecord.streamId = "stream2";
+			splitRecord.value = new Tuple1<Integer>(3);
+		}
+		wrapper.processElement(record);
+		verify(output, times(2)).collect(new StreamRecord<SplitStreamType>(splitRecord, 0));
+	}
+
+	@SuppressWarnings("unchecked")
+	@Test
+	public void testOpen() 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);
+		when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
+		when(taskContext.getTaskName()).thenReturn("name");
+
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+		declarer.declare(new Fields("dummy"));
+		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		final IRichBolt bolt = mock(IRichBolt.class);
+
+		BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
+		wrapper.setup(mock(Output.class), taskContext);
+
+		// test without configuration
+		wrapper.open(null);
+		verify(bolt).prepare(any(Map.class), any(TopologyContext.class), any(OutputCollector.class));
+
+		// test with StormConfig
+		wrapper.open(null);
+		verify(bolt).prepare(same(stormConfig), any(TopologyContext.class),
+				any(OutputCollector.class));
+
+		// test with Configuration
+		final TestDummyBolt testBolt = new TestDummyBolt();
+		wrapper = new BoltWrapper<Object, Object>(testBolt);
+		wrapper.setup(mock(Output.class), taskContext);
+
+		wrapper.open(null);
+		for (Entry<String, String> entry : flinkConfig.toMap().entrySet()) {
+			Assert.assertEquals(entry.getValue(), testBolt.config.get(entry.getKey()));
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	@Test
+	public void testOpenSink() 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);
+		when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
+		when(taskContext.getTaskName()).thenReturn("name");
+
+		final IRichBolt bolt = mock(IRichBolt.class);
+
+		BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
+		wrapper.setup(mock(Output.class), taskContext);
+
+		// test without configuration
+		wrapper.open(null);
+		verify(bolt).prepare(any(Map.class), any(TopologyContext.class),
+				isNull(OutputCollector.class));
+
+		// test with StormConfig
+		wrapper.open(null);
+		verify(bolt).prepare(same(stormConfig), any(TopologyContext.class),
+				isNull(OutputCollector.class));
+
+		// test with Configuration
+		final TestDummyBolt testBolt = new TestDummyBolt();
+		wrapper = new BoltWrapper<Object, Object>(testBolt);
+		wrapper.setup(mock(Output.class), taskContext);
+
+		wrapper.open(null);
+		for (Entry<String, String> entry : flinkConfig.toMap().entrySet()) {
+			Assert.assertEquals(entry.getValue(), testBolt.config.get(entry.getKey()));
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	@Test
+	public void testClose() throws Exception {
+		final IRichBolt bolt = mock(IRichBolt.class);
+
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+		declarer.declare(new Fields("dummy"));
+		PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+		final BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
+
+		final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
+		wrapper.setup(mock(Output.class), taskContext);
+
+		wrapper.close();
+		wrapper.dispose();
+
+		verify(bolt).cleanup();
+	}
+
+	private static final class TestBolt implements IRichBolt {
+		private static final long serialVersionUID = 7278692872260138758L;
+		private OutputCollector collector;
+
+		@SuppressWarnings("rawtypes")
+		@Override
+		public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+			this.collector = collector;
+		}
+
+		int counter = 0;
+		@Override
+		public void execute(backtype.storm.tuple.Tuple input) {
+			if (++counter % 2 == 1) {
+				this.collector.emit("stream1", new Values(input.getInteger(0)));
+			} else {
+				this.collector.emit("stream2", new Values(input.getInteger(0)));
+			}
+		}
+
+		@Override
+		public void cleanup() {}
+
+		@Override
+		public void declareOutputFields(OutputFieldsDeclarer declarer) {
+			declarer.declareStream("stream1", new Fields("a1"));
+			declarer.declareStream("stream2", new Fields("a2"));
+		}
+
+		@Override
+		public Map<String, Object> getComponentConfiguration() {
+			return null;
+		}
+	}
+
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java
new file mode 100644
index 0000000..69d4a8e
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.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.storm.util.AbstractTest;
+import org.apache.flink.storm.wrappers.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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), null, null,
+				null, null, null, null, null, null, null, null, null, null, null, null, null)
+		.setSubscribedState(null, null, null);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java
new file mode 100644
index 0000000..4618101
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+import org.apache.flink.storm.util.AbstractTest;
+import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+
+public class SetupOutputFieldsDeclarerTest extends AbstractTest {
+
+	@Test
+	public void testDeclare() {
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+
+		int numberOfAttributes = this.r.nextInt(26);
+		declarer.declare(createSchema(numberOfAttributes));
+		Assert.assertEquals(1, declarer.outputSchemas.size());
+		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(Utils.DEFAULT_STREAM_ID)
+				.intValue());
+
+		final String sid = "streamId";
+		numberOfAttributes = this.r.nextInt(26);
+		declarer.declareStream(sid, createSchema(numberOfAttributes));
+		Assert.assertEquals(2, declarer.outputSchemas.size());
+		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(sid).intValue());
+	}
+
+	private Fields createSchema(final int numberOfAttributes) {
+		final ArrayList<String> schema = new ArrayList<String>(numberOfAttributes);
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			schema.add("a" + i);
+		}
+		return new Fields(schema);
+	}
+
+	@Test
+	public void testDeclareDirect() {
+		new SetupOutputFieldsDeclarer().declare(false, new Fields());
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareDirectFail() {
+		new SetupOutputFieldsDeclarer().declare(true, new Fields());
+	}
+
+	@Test
+	public void testDeclareStream() {
+		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, new Fields());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareStreamFail() {
+		new SetupOutputFieldsDeclarer().declareStream(null, new Fields());
+	}
+
+	@Test
+	public void testDeclareFullStream() {
+		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareFullStreamFailNonDefaultStream() {
+		new SetupOutputFieldsDeclarer().declareStream(null, false, new Fields());
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareFullStreamFailDirect() {
+		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, new Fields());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java
new file mode 100644
index 0000000..6b60d2b
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.tuple.Values;
+
+import org.apache.flink.api.java.tuple.Tuple;
+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;
+
+import java.util.HashMap;
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+public class SpoutCollectorTest extends AbstractTest {
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	@Test
+	public void testSpoutStormCollector() throws InstantiationException, IllegalAccessException {
+		for (int numberOfAttributes = -1; numberOfAttributes < 26; ++numberOfAttributes) {
+			final SourceContext flinkCollector = mock(SourceContext.class);
+			Tuple flinkTuple = null;
+			final Values tuple = new Values();
+
+			SpoutCollector<?> collector;
+
+			final String streamId = "streamId";
+			HashMap<String, Integer> attributes = new HashMap<String, Integer>();
+			attributes.put(streamId, numberOfAttributes);
+
+			if (numberOfAttributes == -1) {
+				collector = new SpoutCollector(attributes, flinkCollector);
+				tuple.add(new Integer(this.r.nextInt()));
+
+			} else {
+				collector = new SpoutCollector(attributes, flinkCollector);
+				flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
+
+				for (int i = 0; i < numberOfAttributes; ++i) {
+					tuple.add(new Integer(this.r.nextInt()));
+					flinkTuple.setField(tuple.get(i), i);
+				}
+			}
+
+			final List<Integer> taskIds;
+			final Object messageId = new Integer(this.r.nextInt());
+
+			taskIds = collector.emit(streamId, tuple, messageId);
+
+			Assert.assertNull(taskIds);
+
+			if (numberOfAttributes == -1) {
+				verify(flinkCollector).collect(tuple.get(0));
+			} else {
+				verify(flinkCollector).collect(flinkTuple);
+			}
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	@Test(expected = UnsupportedOperationException.class)
+	public void testEmitDirect() {
+		new SpoutCollector<Object>(mock(HashMap.class), mock(SourceContext.class)).emitDirect(
+				0, null, null,
+				(Object) null);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java
----------------------------------------------------------------------
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 0000000..227d736
--- /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<String, String> 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<Object>(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<Tuple1<Integer>> expectedResult = new LinkedList<Tuple1<Integer>>();
+		for (int i = numberOfCalls - 1; i >= 0; --i) {
+			expectedResult.add(new Tuple1<Integer>(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<Tuple1<Integer>> spoutWrapper = new SpoutWrapper<Tuple1<Integer>>(
+				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<Object>(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<Object>(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<Tuple1<Integer>> spoutWrapper = new SpoutWrapper<Tuple1<Integer>>(spout);
+		spoutWrapper.setRuntimeContext(taskContext);
+
+		spoutWrapper.cancel();
+		final TestContext collector = new TestContext();
+		spoutWrapper.run(collector);
+
+		Assert.assertEquals(new LinkedList<Tuple1<Integer>>(), collector.result);
+	}
+
+	@Test
+	public void testClose() throws Exception {
+		final IRichSpout spout = mock(IRichSpout.class);
+		final SpoutWrapper<Tuple1<Integer>> spoutWrapper = new SpoutWrapper<Tuple1<Integer>>(spout);
+
+		spoutWrapper.close();
+
+		verify(spout).close();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java
new file mode 100644
index 0000000..155fcd9
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java
@@ -0,0 +1,660 @@
+/*
+ * 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 org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple5;
+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;
+
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+
+public class StormTupleTest extends AbstractTest {
+	private static final String fieldName = "fieldName";
+	private static final String fieldNamePojo = "member";
+
+	private int arity, index;
+
+	@Override
+	@Before
+	public void prepare() {
+		super.prepare();
+		this.arity = 1 + r.nextInt(25);
+		this.index = r.nextInt(this.arity);
+	}
+
+	@Test
+	public void nonTupleTest() {
+		final Object flinkTuple = this.r.nextInt();
+
+		final StormTuple<Object> tuple = new StormTuple<Object>(flinkTuple, null);
+		Assert.assertSame(flinkTuple, tuple.getValue(0));
+
+		final List<Object> values = tuple.getValues();
+		Assert.assertEquals(1, values.size());
+		Assert.assertEquals(flinkTuple, values.get(0));
+	}
+
+	@Test
+	public void tupleTest() throws InstantiationException, IllegalAccessException {
+		final int numberOfAttributes = this.r.nextInt(26);
+		final Object[] data = new Object[numberOfAttributes];
+
+		final Tuple flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			data[i] = this.r.nextInt();
+			flinkTuple.setField(data[i], i);
+		}
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		final List<Object> values = tuple.getValues();
+
+		Assert.assertEquals(numberOfAttributes, values.size());
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			Assert.assertEquals(flinkTuple.getField(i), values.get(i));
+		}
+
+		Assert.assertEquals(numberOfAttributes, tuple.size());
+	}
+
+	@Test
+	public void testBinary() {
+		final byte[] data = new byte[this.r.nextInt(15)];
+		this.r.nextBytes(data);
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getBinary(index));
+	}
+
+	@Test
+	public void testBoolean() {
+		final Boolean flinkTuple = this.r.nextBoolean();
+
+		final StormTuple<Boolean> tuple = new StormTuple<Boolean>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getBoolean(0));
+	}
+
+	@Test
+	public void testByte() {
+		final Byte flinkTuple = (byte) this.r.nextInt();
+
+		final StormTuple<Byte> tuple = new StormTuple<Byte>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getByte(0));
+	}
+
+	@Test
+	public void testDouble() {
+		final Double flinkTuple = this.r.nextDouble();
+
+		final StormTuple<Double> tuple = new StormTuple<Double>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getDouble(0));
+	}
+
+	@Test
+	public void testFloat() {
+		final Float flinkTuple = this.r.nextFloat();
+
+		final StormTuple<Float> tuple = new StormTuple<Float>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getFloat(0));
+	}
+
+	@Test
+	public void testInteger() {
+		final Integer flinkTuple = this.r.nextInt();
+
+		final StormTuple<Integer> tuple = new StormTuple<Integer>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getInteger(0));
+	}
+
+	@Test
+	public void testLong() {
+		final Long flinkTuple = this.r.nextLong();
+
+		final StormTuple<Long> tuple = new StormTuple<Long>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getLong(0));
+	}
+
+	@Test
+	public void testShort() {
+		final Short flinkTuple = (short) this.r.nextInt();
+
+		final StormTuple<Short> tuple = new StormTuple<Short>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getShort(0));
+	}
+
+	@Test
+	public void testString() {
+		final byte[] data = new byte[this.r.nextInt(15)];
+		this.r.nextBytes(data);
+		final String flinkTuple = new String(data);
+
+		final StormTuple<String> tuple = new StormTuple<String>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple, tuple.getString(0));
+	}
+
+	@Test
+	public void testBinaryTuple() {
+		final byte[] data = new byte[this.r.nextInt(15)];
+		this.r.nextBytes(data);
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getBinary(index));
+	}
+
+	@Test
+	public void testBooleanTuple() {
+		final Boolean data = this.r.nextBoolean();
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getBoolean(index));
+	}
+
+	@Test
+	public void testByteTuple() {
+		final Byte data = (byte) this.r.nextInt();
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getByte(index));
+	}
+
+	@Test
+	public void testDoubleTuple() {
+		final Double data = this.r.nextDouble();
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getDouble(index));
+	}
+
+	@Test
+	public void testFloatTuple() {
+		final Float data = this.r.nextFloat();
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getFloat(index));
+	}
+
+	@Test
+	public void testIntegerTuple() {
+		final Integer data = this.r.nextInt();
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getInteger(index));
+	}
+
+	@Test
+	public void testLongTuple() {
+		final Long data = this.r.nextLong();
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getLong(index));
+	}
+
+	@Test
+	public void testShortTuple() {
+		final Short data = (short) this.r.nextInt();
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getShort(index));
+	}
+
+	@Test
+	public void testStringTuple() {
+		final byte[] rawdata = new byte[this.r.nextInt(15)];
+		this.r.nextBytes(rawdata);
+		final String data = new String(rawdata);
+
+		final int index = this.r.nextInt(5);
+		final Tuple flinkTuple = new Tuple5<Object, Object, Object, Object, Object>();
+		flinkTuple.setField(data, index);
+
+		final StormTuple<Tuple> tuple = new StormTuple<Tuple>(flinkTuple, null);
+		Assert.assertEquals(flinkTuple.getField(index), tuple.getString(index));
+	}
+
+	@Test
+	public void testContains() throws Exception {
+		Fields schema = new Fields("a1", "a2");
+		StormTuple<Object> tuple = new StormTuple<Object>(Tuple.getTupleClass(1).newInstance(),
+				schema);
+
+		Assert.assertTrue(tuple.contains("a1"));
+		Assert.assertTrue(tuple.contains("a2"));
+		Assert.assertFalse(tuple.contains("a3"));
+	}
+
+	@Test
+	public void testGetFields() throws Exception {
+		Fields schema = new Fields();
+
+		Assert.assertSame(schema, new StormTuple<Object>(Tuple.getTupleClass(1).newInstance(),
+				schema).getFields());
+
+		Assert.assertSame(null, new StormTuple<Object>(null, schema).getFields());
+
+	}
+
+	@Test
+	public void testFieldIndex() throws Exception {
+		Fields schema = new Fields("a1", "a2");
+		StormTuple<Object> tuple = new StormTuple<Object>(Tuple.getTupleClass(1).newInstance(),
+				schema);
+
+		Assert.assertEquals(0, tuple.fieldIndex("a1"));
+		Assert.assertEquals(1, tuple.fieldIndex("a2"));
+	}
+
+	@SuppressWarnings({ "rawtypes", "unchecked" })
+	@Test
+	public void testSelect() throws Exception {
+		Tuple tuple = Tuple.getTupleClass(arity).newInstance();
+		Values values = new Values();
+
+		ArrayList<String> attributeNames = new ArrayList<String>(arity);
+		ArrayList<String> filterNames = new ArrayList<String>(arity);
+
+		for (int i = 0; i < arity; ++i) {
+			tuple.setField(i, i);
+			attributeNames.add("a" + i);
+
+			if (r.nextBoolean()) {
+				filterNames.add("a" + i);
+				values.add(i);
+			}
+		}
+		Fields schema = new Fields(attributeNames);
+		Fields selector = new Fields(filterNames);
+
+		Assert.assertEquals(values, new StormTuple(tuple, schema).select(selector));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetValueByField() throws Exception {
+		Object value = mock(Object.class);
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getValueByField(fieldName));
+
+	}
+
+	@Test
+	public void testGetValueByFieldPojo() throws Exception {
+		Object value = mock(Object.class);
+		TestPojoMember<Object> pojo = new TestPojoMember<Object>(value);
+		StormTuple<TestPojoMember<Object>> tuple = new StormTuple<TestPojoMember<Object>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getValueByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetValueByFieldPojoGetter() throws Exception {
+		Object value = mock(Object.class);
+		TestPojoGetter<Object> pojo = new TestPojoGetter<Object>(value);
+		StormTuple<TestPojoGetter<Object>> tuple = new StormTuple<TestPojoGetter<Object>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getValueByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetStringByField() throws Exception {
+		String value = "stringValue";
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getStringByField(fieldName));
+	}
+
+	@Test
+	public void testGetStringByFieldPojo() throws Exception {
+		String value = "stringValue";
+		TestPojoMember<String> pojo = new TestPojoMember<String>(value);
+		StormTuple<TestPojoMember<String>> tuple = new StormTuple<TestPojoMember<String>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getStringByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetStringByFieldPojoGetter() throws Exception {
+		String value = "stringValue";
+		TestPojoGetter<String> pojo = new TestPojoGetter<String>(value);
+		StormTuple<TestPojoGetter<String>> tuple = new StormTuple<TestPojoGetter<String>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getStringByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetIntegerByField() throws Exception {
+		Integer value = r.nextInt();
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getIntegerByField(fieldName));
+	}
+
+	@Test
+	public void testGetIntegerByFieldPojo() throws Exception {
+		Integer value = r.nextInt();
+		TestPojoMember<Integer> pojo = new TestPojoMember<Integer>(value);
+		StormTuple<TestPojoMember<Integer>> tuple = new StormTuple<TestPojoMember<Integer>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getIntegerByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetIntegerByFieldPojoGetter() throws Exception {
+		Integer value = r.nextInt();
+		TestPojoGetter<Integer> pojo = new TestPojoGetter<Integer>(value);
+		StormTuple<TestPojoGetter<Integer>> tuple = new StormTuple<TestPojoGetter<Integer>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getIntegerByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetLongByField() throws Exception {
+		Long value = r.nextLong();
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getLongByField(fieldName));
+	}
+
+	@Test
+	public void testGetLongByFieldPojo() throws Exception {
+		Long value = r.nextLong();
+		TestPojoMember<Long> pojo = new TestPojoMember<Long>(value);
+		StormTuple<TestPojoMember<Long>> tuple = new StormTuple<TestPojoMember<Long>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getLongByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetLongByFieldPojoGetter() throws Exception {
+		Long value = r.nextLong();
+		TestPojoGetter<Long> pojo = new TestPojoGetter<Long>(value);
+		StormTuple<TestPojoGetter<Long>> tuple = new StormTuple<TestPojoGetter<Long>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getLongByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetBooleanByField() throws Exception {
+		Boolean value = r.nextBoolean();
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertEquals(value, tuple.getBooleanByField(fieldName));
+	}
+
+	@Test
+	public void testGetBooleanByFieldPojo() throws Exception {
+		Boolean value = r.nextBoolean();
+		TestPojoMember<Boolean> pojo = new TestPojoMember<Boolean>(value);
+		StormTuple<TestPojoMember<Boolean>> tuple = new StormTuple<TestPojoMember<Boolean>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getBooleanByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetBooleanByFieldPojoGetter() throws Exception {
+		Boolean value = r.nextBoolean();
+		TestPojoGetter<Boolean> pojo = new TestPojoGetter<Boolean>(value);
+		StormTuple<TestPojoGetter<Boolean>> tuple = new StormTuple<TestPojoGetter<Boolean>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getBooleanByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetShortByField() throws Exception {
+		Short value = (short) r.nextInt();
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getShortByField(fieldName));
+	}
+
+	@Test
+	public void testGetShortByFieldPojo() throws Exception {
+		Short value = (short) r.nextInt();
+		TestPojoMember<Short> pojo = new TestPojoMember<Short>(value);
+		StormTuple<TestPojoMember<Short>> tuple = new StormTuple<TestPojoMember<Short>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getShortByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetShortByFieldPojoGetter() throws Exception {
+		Short value = (short) r.nextInt();
+		TestPojoGetter<Short> pojo = new TestPojoGetter<Short>(value);
+		StormTuple<TestPojoGetter<Short>> tuple = new StormTuple<TestPojoGetter<Short>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getShortByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetByteByField() throws Exception {
+		Byte value = new Byte((byte) r.nextInt());
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getByteByField(fieldName));
+	}
+
+	@Test
+	public void testGetByteByFieldPojo() throws Exception {
+		Byte value = new Byte((byte) r.nextInt());
+		TestPojoMember<Byte> pojo = new TestPojoMember<Byte>(value);
+		StormTuple<TestPojoMember<Byte>> tuple = new StormTuple<TestPojoMember<Byte>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getByteByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetByteByFieldPojoGetter() throws Exception {
+		Byte value = new Byte((byte) r.nextInt());
+		TestPojoGetter<Byte> pojo = new TestPojoGetter<Byte>(value);
+		StormTuple<TestPojoGetter<Byte>> tuple = new StormTuple<TestPojoGetter<Byte>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getByteByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetDoubleByField() throws Exception {
+		Double value = r.nextDouble();
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getDoubleByField(fieldName));
+	}
+
+	@Test
+	public void testGetDoubleByFieldPojo() throws Exception {
+		Double value = r.nextDouble();
+		TestPojoMember<Double> pojo = new TestPojoMember<Double>(value);
+		StormTuple<TestPojoMember<Double>> tuple = new StormTuple<TestPojoMember<Double>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getDoubleByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetDoubleByFieldPojoGetter() throws Exception {
+		Double value = r.nextDouble();
+		TestPojoGetter<Double> pojo = new TestPojoGetter<Double>(value);
+		StormTuple<TestPojoGetter<Double>> tuple = new StormTuple<TestPojoGetter<Double>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getDoubleByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetFloatByField() throws Exception {
+		Float value = r.nextFloat();
+		StormTuple tuple = testGetByField(arity, index, value);
+		Assert.assertSame(value, tuple.getFloatByField(fieldName));
+	}
+
+	@Test
+	public void testGetFloatByFieldPojo() throws Exception {
+		Float value = r.nextFloat();
+		TestPojoMember<Float> pojo = new TestPojoMember<Float>(value);
+		StormTuple<TestPojoMember<Float>> tuple = new StormTuple<TestPojoMember<Float>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getFloatByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetFloatByFieldPojoGetter() throws Exception {
+		Float value = r.nextFloat();
+		TestPojoGetter<Float> pojo = new TestPojoGetter<Float>(value);
+		StormTuple<TestPojoGetter<Float>> tuple = new StormTuple<TestPojoGetter<Float>>(pojo,
+				null);
+		Assert.assertSame(value, tuple.getFloatByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testGetBinaryByField() throws Exception {
+		byte[] data = new byte[1 + r.nextInt(20)];
+		r.nextBytes(data);
+		StormTuple tuple = testGetByField(arity, index, data);
+		Assert.assertSame(data, tuple.getBinaryByField(fieldName));
+	}
+
+	@Test
+	public void testGetBinaryFieldPojo() throws Exception {
+		byte[] data = new byte[1 + r.nextInt(20)];
+		r.nextBytes(data);
+		TestPojoMember<byte[]> pojo = new TestPojoMember<byte[]>(data);
+		StormTuple<TestPojoMember<byte[]>> tuple = new StormTuple<TestPojoMember<byte[]>>(pojo,
+				null);
+		Assert.assertSame(data, tuple.getBinaryByField(fieldNamePojo));
+	}
+
+	@Test
+	public void testGetBinaryByFieldPojoGetter() throws Exception {
+		byte[] data = new byte[1 + r.nextInt(20)];
+		r.nextBytes(data);
+		TestPojoGetter<byte[]> pojo = new TestPojoGetter<byte[]>(data);
+		StormTuple<TestPojoGetter<byte[]>> tuple = new StormTuple<TestPojoGetter<byte[]>>(pojo,
+				null);
+		Assert.assertSame(data, tuple.getBinaryByField(fieldNamePojo));
+	}
+
+	@SuppressWarnings({ "unchecked", "rawtypes" })
+	private <T> StormTuple testGetByField(int arity, int index, T value)
+			throws Exception {
+
+		assert (index < arity);
+
+		Tuple tuple = Tuple.getTupleClass(arity).newInstance();
+		tuple.setField(value, index);
+
+		ArrayList<String> attributeNames = new ArrayList<String>(arity);
+		for(int i = 0; i < arity; ++i) {
+			if(i == index) {
+				attributeNames.add(fieldName);
+			} else {
+				attributeNames.add("" + i);
+			}
+		}
+		Fields schema = new Fields(attributeNames);
+
+		return new StormTuple(tuple, schema);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testGetSourceGlobalStreamid() {
+		new StormTuple<Object>(null, null).getSourceGlobalStreamid();
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testGetSourceComponent() {
+		new StormTuple<Object>(null, null).getSourceComponent();
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testGetSourceTask() {
+		new StormTuple<Object>(null, null).getSourceTask();
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testGetSourceStreamId() {
+		new StormTuple<Object>(null, null).getSourceStreamId();
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testGetMessageId() {
+		new StormTuple<Object>(null, null).getMessageId();
+	}
+
+	public static class TestPojoMember<T> {
+		public T member;
+
+		public TestPojoMember(T value) {
+			this.member = value;
+		}
+	}
+
+	public static class TestPojoGetter<T> {
+		private T member;
+
+		public TestPojoGetter(T value) {
+			this.member = value;
+		}
+
+		public T getMember() {
+			return this.member;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/TestContext.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/TestContext.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/TestContext.java
new file mode 100644
index 0000000..4c4749a
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/TestContext.java
@@ -0,0 +1,56 @@
+/*
+ * 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 org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+import java.util.LinkedList;
+
+class TestContext implements SourceContext<Tuple1<Integer>> {
+	public LinkedList<Tuple1<Integer>> result = new LinkedList<Tuple1<Integer>>();
+
+	public TestContext() {
+	}
+
+	@Override
+	public void collect(final Tuple1<Integer> record) {
+		this.result.add(record.copy());
+	}
+
+	@Override
+	public void collectWithTimestamp(Tuple1<Integer> element, long timestamp) {
+		this.result.add(element.copy());
+	}
+
+	@Override
+	public void emitWatermark(Watermark mark) {
+		// ignore it
+	}
+
+	@Override
+	public Object getCheckpointLock() {
+		return null;
+	}
+
+	@Override
+	public void close() {
+
+	}
+}


[05/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java
new file mode 100644
index 0000000..71a5e8d
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java
@@ -0,0 +1,77 @@
+/*
+ * 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.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.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 WordCountLocal} all bolts access the field of
+ * input tuples by name instead of index.
+ * <p/>
+ * This example shows how to run program directly within Java, thus it cannot be used to submit a {@link StormTopology}
+ * via Flink command line clients (ie, bin/flink).
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCountLocalByName &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>run a regular Storm program locally on Flink
+ * </ul>
+ */
+public class WordCountLocalByName {
+	public final static String topologyId = "Storm WordCountName";
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!WordCountTopology.parseParameters(args)) {
+			return;
+		}
+
+		// build Topology the Storm way
+		final FlinkTopologyBuilder builder = WordCountTopology.buildTopology(false);
+
+		// execute program locally
+		final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster();
+		cluster.submitTopology(topologyId, null, builder.createTopology());
+
+		Utils.sleep(10 * 1000);
+
+		// TODO kill does no do anything so far
+		cluster.killTopology(topologyId);
+		cluster.shutdown();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java
new file mode 100644
index 0000000..2e4fb03
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java
@@ -0,0 +1,86 @@
+/*
+ * 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.wordcount;
+
+import backtype.storm.Config;
+import backtype.storm.generated.AlreadyAliveException;
+import backtype.storm.generated.InvalidTopologyException;
+import backtype.storm.generated.NotAliveException;
+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.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
+ * 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.
+ * <p/>
+ * 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).
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCountRemoteByClient &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>submit a regular Storm program to a local or remote Flink cluster.</li>
+ * </ul>
+ */
+public class WordCountRemoteByClient {
+	public final static String topologyId = "Storm WordCount";
+	private final static String uploadedJarLocation = "target/WordCount-StormTopology.jar";
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws AlreadyAliveException, InvalidTopologyException,
+	NotAliveException {
+
+		if (!WordCountTopology.parseParameters(args)) {
+			return;
+		}
+
+		// build Topology the Storm way
+		final FlinkTopologyBuilder builder = WordCountTopology.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);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java
new file mode 100644
index 0000000..173074c
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java
@@ -0,0 +1,84 @@
+/*
+ * 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.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.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
+ * 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.
+ * <p/>
+ * This example shows how to submit the program via Java as well as Flink's command line client (ie, bin/flink).
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCountRemoteBySubmitter &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>submit a regular Storm program to a local or remote Flink cluster.</li>
+ * </ul>
+ */
+public class WordCountRemoteBySubmitter {
+	public final static String topologyId = "Storm WordCount";
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!WordCountTopology.parseParameters(args)) {
+			return;
+		}
+
+		// build Topology the Storm way
+		final FlinkTopologyBuilder builder = WordCountTopology.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/WordCount-StormTopology.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);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java
new file mode 100644
index 0000000..8ee374d
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java
@@ -0,0 +1,136 @@
+/*
+ * 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.wordcount;
+
+import backtype.storm.generated.StormTopology;
+import backtype.storm.tuple.Fields;
+
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+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
+ * fashion. The program is constructed as a regular {@link StormTopology}.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage:
+ * <code>WordCount[Local|LocalByName|RemoteByClient|RemoteBySubmitter] &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>how to construct a regular Storm topology as Flink program</li>
+ * </ul>
+ */
+public class WordCountTopology {
+	public final static String spoutId = "source";
+	public final static String tokenierzerId = "tokenizer";
+	public final static String counterId = "counter";
+	public final static String sinkId = "sink";
+	private final static OutputFormatter formatter = new TupleOutputFormatter();
+
+	public static FlinkTopologyBuilder buildTopology() {
+		return buildTopology(true);
+	}
+
+	public static FlinkTopologyBuilder buildTopology(boolean indexOrName) {
+
+		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
+
+		// get input data
+		if (fileInputOutput) {
+			// read the text file from given input path
+			final String[] tokens = textPath.split(":");
+			final String inputFile = tokens[tokens.length - 1];
+			builder.setSpout(spoutId, new WordCountFileSpout(inputFile));
+		} else {
+			builder.setSpout(spoutId, new WordCountInMemorySpout());
+		}
+
+		if (indexOrName) {
+			// split up the lines in pairs (2-tuples) containing: (word,1)
+			builder.setBolt(tokenierzerId, new BoltTokenizer(), 4).shuffleGrouping(spoutId);
+			// group by the tuple field "0" and sum up tuple field "1"
+			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 BoltTokenizerByName(), 4).shuffleGrouping(
+					spoutId);
+			// group by the tuple field "0" and sum up tuple field "1"
+			builder.setBolt(counterId, new BoltCounterByName(), 4).fieldsGrouping(
+					tokenierzerId, new Fields(BoltTokenizerByName.ATTRIBUTE_WORD));
+		}
+
+		// 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 BoltFileSink(outputFile, formatter)).shuffleGrouping(counterId);
+		} else {
+			builder.setBolt(sinkId, new BoltPrintSink(formatter), 4).shuffleGrouping(counterId);
+		}
+
+		return builder;
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileInputOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileInputOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: WordCount* <text path> <result path>");
+				return false;
+			}
+		} else {
+			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: WordCount* <text path> <result path>");
+		}
+
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java
new file mode 100644
index 0000000..d21a584
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java
@@ -0,0 +1,90 @@
+/*
+ * 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.wordcount.operators;
+
+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.tuple.Values;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Implements the word counter that counts the occurrence of each unique word. The bolt takes a pair (input tuple
+ * schema: {@code <String,Integer>}) and sums the given word count for each unique word (output tuple schema:
+ * {@code <String,Integer>} ).
+ * <p>
+ * Same as {@link BoltCounterByName}, but accesses input attribute by index (instead of name).
+ */
+public class BoltCounter implements IRichBolt {
+	private static final long serialVersionUID = 399619605462625934L;
+
+	public static final String ATTRIBUTE_WORD = "word";
+	public static final String ATTRIBUTE_COUNT = "count";
+
+	private final HashMap<String, Count> counts = new HashMap<String, Count>();
+	private OutputCollector collector;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void execute(final Tuple input) {
+		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.getInteger(BoltTokenizer.ATTRIBUTE_COUNT_INDEX);
+
+		this.collector.emit(new Values(word, currentCount.count));
+	}
+
+	@Override
+	public void cleanup() {/* nothing to do */}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+	/**
+	 * A counter helper to emit immutable tuples to the given stormCollector and avoid unnecessary object
+	 * creating/deletion.
+	 */
+	private static final class Count {
+		public int count;
+
+		public Count() {/* nothing to do */}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java
new file mode 100644
index 0000000..d5c05d7
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java
@@ -0,0 +1,90 @@
+/*
+ * 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.wordcount.operators;
+
+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.tuple.Values;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Implements the word counter that counts the occurrence of each unique word. The bolt takes a pair (input tuple
+ * schema: {@code <String,Integer>}) and sums the given word count for each unique word (output tuple schema:
+ * {@code <String,Integer>} ).
+ * <p>
+ * Same as {@link BoltCounter}, but accesses input attribute by name (instead of index).
+ */
+public class BoltCounterByName implements IRichBolt {
+	private static final long serialVersionUID = 399619605462625934L;
+
+	public static final String ATTRIBUTE_WORD = "word";
+	public static final String ATTRIBUTE_COUNT = "count";
+
+	private final HashMap<String, Count> counts = new HashMap<String, Count>();
+	private OutputCollector collector;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void execute(final Tuple input) {
+		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.getIntegerByField(BoltTokenizer.ATTRIBUTE_COUNT);
+
+		this.collector.emit(new Values(word, currentCount.count));
+	}
+
+	@Override
+	public void cleanup() {/* nothing to do */}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+	/**
+	 * A counter helper to emit immutable tuples to the given stormCollector and avoid unnecessary object
+	 * creating/deletion.
+	 */
+	private static final class Count {
+		public int count;
+
+		public Count() {/* nothing to do */}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java
new file mode 100644
index 0000000..74d6a99
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java
@@ -0,0 +1,78 @@
+/*
+ * 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.wordcount.operators;
+
+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.tuple.Values;
+
+import java.util.Map;
+
+/**
+ * Implements the string tokenizer that splits sentences into words as a bolt. The bolt takes a line (input tuple
+ * schema: {@code <String>}) and splits it into multiple pairs in the form of "(word,1)" (output tuple schema:
+ * {@code <String,Integer>}).
+ * <p>
+ * Same as {@link BoltTokenizerByName}, but accesses input attribute by index (instead of name).
+ */
+public final class BoltTokenizer implements IRichBolt {
+	private static final long serialVersionUID = -8589620297208175149L;
+
+	public static final String ATTRIBUTE_WORD = "word";
+	public static final String ATTRIBUTE_COUNT = "count";
+
+	public static final int ATTRIBUTE_WORD_INDEX = 0;
+	public static final int ATTRIBUTE_COUNT_INDEX = 1;
+
+	private OutputCollector collector;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void execute(final Tuple input) {
+		final String[] tokens = input.getString(0).toLowerCase().split("\\W+");
+
+		for (final String token : tokens) {
+			if (token.length() > 0) {
+				this.collector.emit(new Values(token, 1));
+			}
+		}
+	}
+
+	@Override
+	public void cleanup() {/* nothing to do */}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java
new file mode 100644
index 0000000..3c56b36
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java
@@ -0,0 +1,78 @@
+/*
+ * 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.wordcount.operators;
+
+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.tuple.Values;
+
+import java.util.Map;
+
+/**
+ * Implements the string tokenizer that splits sentences into words as a bolt. The bolt takes a line (input tuple
+ * schema: {@code <String>}) and splits it into multiple pairs in the form of "(word,1)" (output tuple schema:
+ * {@code <String,Integer>}).
+ * <p>
+ * Same as {@link BoltTokenizer}, but accesses input attribute by name (instead of index).
+ */
+public final class BoltTokenizerByName implements IRichBolt {
+	private static final long serialVersionUID = -8589620297208175149L;
+
+	public static final String ATTRIBUTE_WORD = "word";
+	public static final String ATTRIBUTE_COUNT = "count";
+
+	public static final int ATTRIBUTE_WORD_INDEX = 0;
+	public static final int ATTRIBUTE_COUNT_INDEX = 1;
+
+	private OutputCollector collector;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void execute(final Tuple input) {
+		final String[] tokens = input.getStringByField("sentence").toLowerCase().split("\\W+");
+
+		for (final String token : tokens) {
+			if (token.length() > 0) {
+				this.collector.emit(new Values(token, 1));
+			}
+		}
+	}
+
+	@Override
+	public void cleanup() {/* nothing to do */}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataPojos.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataPojos.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataPojos.java
new file mode 100644
index 0000000..3a8fd3a
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataPojos.java
@@ -0,0 +1,59 @@
+/*
+ * 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.wordcount.operators;
+
+import java.io.Serializable;
+
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+
+public class WordCountDataPojos {
+	public static Sentence[] SENTENCES;
+
+	static {
+		SENTENCES = new Sentence[WordCountData.WORDS.length];
+		for (int i = 0; i < SENTENCES.length; ++i) {
+			SENTENCES[i] = new Sentence(WordCountData.WORDS[i]);
+		}
+	}
+
+	public static class Sentence implements Serializable {
+		private static final long serialVersionUID = -7336372859203407522L;
+
+		private String sentence;
+
+		public Sentence() {
+		}
+
+		public Sentence(String sentence) {
+			this.sentence = sentence;
+		}
+
+		public String getSentence() {
+			return sentence;
+		}
+
+		public void setSentence(String sentence) {
+			this.sentence = sentence;
+		}
+
+		@Override
+		public String toString() {
+			return "(" + this.sentence + ")";
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataTuple.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataTuple.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataTuple.java
new file mode 100644
index 0000000..16e2ba0
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountDataTuple.java
@@ -0,0 +1,34 @@
+/*
+ * 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.wordcount.operators;
+
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+
+@SuppressWarnings("unchecked")
+public class WordCountDataTuple {
+	public static Tuple1<String>[] TUPLES;
+
+	static {
+		TUPLES = new Tuple1[WordCountData.WORDS.length];
+		for (int i = 0; i < TUPLES.length; ++i) {
+			TUPLES[i] = new Tuple1<String>(WordCountData.WORDS[i]);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java
new file mode 100644
index 0000000..76a198f
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java
@@ -0,0 +1,39 @@
+/*
+ * 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.wordcount.operators;
+
+import org.apache.flink.storm.util.FileSpout;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+
+/**
+ * Implements a Spout that reads data from a given local file.
+ */
+public final class WordCountFileSpout extends FileSpout {
+	private static final long serialVersionUID = 2372251989250954503L;
+
+	public WordCountFileSpout(String path) {
+		super(path);
+	}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields("sentence"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java
new file mode 100644
index 0000000..eb96160
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java
@@ -0,0 +1,40 @@
+/*
+ * 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.wordcount.operators;
+
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+import org.apache.flink.storm.util.InMemorySpout;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+
+/**
+ * Implements a Spout that reads data from {@link WordCountData#WORDS}.
+ */
+public final class WordCountInMemorySpout extends InMemorySpout<String> {
+	private static final long serialVersionUID = 8832143302409465843L;
+
+	public WordCountInMemorySpout() {
+		super(WordCountData.WORDS);
+	}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields("sentence"));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java
new file mode 100644
index 0000000..781396c
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithBoltITCase.java
@@ -0,0 +1,49 @@
+/*
+ * 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.exclamation;
+
+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 ExclamationWithBoltITCase extends StormTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+	protected String exclamationNum;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+		this.exclamationNum = "3";
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(ExclamationData.TEXT_WITH_EXCLAMATIONS, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExclamationWithBolt.main(new String[]{this.textPath, this.resultPath, this.exclamationNum});
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.java
new file mode 100644
index 0000000..36b8aed
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/ExclamationWithSpoutITCase.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.storm.exclamation;
+
+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 ExclamationWithSpoutITCase extends StormTestBase {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(ExclamationData.TEXT_WITH_EXCLAMATIONS, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExclamationWithSpout.main(new String[]{this.textPath, this.resultPath});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/StormExclamationLocalITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/StormExclamationLocalITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/StormExclamationLocalITCase.java
new file mode 100644
index 0000000..cec276f
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/StormExclamationLocalITCase.java
@@ -0,0 +1,48 @@
+/*
+ * 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.exclamation;
+
+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 {
+
+	protected String textPath;
+	protected String resultPath;
+	protected String exclamationNum;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+		this.exclamationNum = "3";
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(ExclamationData.TEXT_WITH_EXCLAMATIONS, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		ExclamationLocal.main(new String[]{this.textPath, this.resultPath, this.exclamationNum});
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/util/ExclamationData.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/util/ExclamationData.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/util/ExclamationData.java
new file mode 100644
index 0000000..3c435f9
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/exclamation/util/ExclamationData.java
@@ -0,0 +1,98 @@
+/*
+ * 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.exclamation.util;
+
+public class ExclamationData {
+
+	public static final String TEXT_WITH_EXCLAMATIONS =
+			"Goethe - Faust: Der Tragoedie erster Teil!!!!!!\n"
+					+ "Prolog im Himmel.!!!!!!\n"
+					+ "Der Herr. Die himmlischen Heerscharen. Nachher Mephistopheles. Die drei!!!!!!\n"
+					+ "Erzengel treten vor.!!!!!!\n"
+					+ "RAPHAEL: Die Sonne toent, nach alter Weise, In Brudersphaeren Wettgesang,!!!!!!\n"
+					+ "Und ihre vorgeschriebne Reise Vollendet sie mit Donnergang. Ihr Anblick!!!!!!\n"
+					+ "gibt den Engeln Staerke, Wenn keiner Sie ergruenden mag; die unbegreiflich!!!!!!\n"
+					+ "hohen Werke Sind herrlich wie am ersten Tag.!!!!!!\n"
+					+ "GABRIEL: Und schnell und unbegreiflich schnelle Dreht sich umher der Erde!!!!!!\n"
+					+ "Pracht; Es wechselt Paradieseshelle Mit tiefer, schauervoller Nacht. Es!!!!!!\n"
+					+ "schaeumt das Meer in breiten Fluessen Am tiefen Grund der Felsen auf, Und!!!!!!\n"
+					+ "Fels und Meer wird fortgerissen Im ewig schnellem Sphaerenlauf.!!!!!!\n"
+					+ "MICHAEL: Und Stuerme brausen um die Wette Vom Meer aufs Land, vom Land!!!!!!\n"
+					+ "aufs Meer, und bilden wuetend eine Kette Der tiefsten Wirkung rings umher.!!!!!!\n"
+					+ "Da flammt ein blitzendes Verheeren Dem Pfade vor des Donnerschlags. Doch!!!!!!\n"
+					+ "deine Boten, Herr, verehren Das sanfte Wandeln deines Tags.!!!!!!\n"
+					+ "ZU DREI: Der Anblick gibt den Engeln Staerke, Da keiner dich ergruenden!!!!!!\n"
+					+ "mag, Und alle deine hohen Werke Sind herrlich wie am ersten Tag.!!!!!!\n"
+					+ "MEPHISTOPHELES: Da du, o Herr, dich einmal wieder nahst Und fragst, wie!!!!!!\n"
+					+ "alles sich bei uns befinde, Und du mich sonst gewoehnlich gerne sahst, So!!!!!!\n"
+					+ "siehst du mich auch unter dem Gesinde. Verzeih, ich kann nicht hohe Worte!!!!!!\n"
+					+ "machen, Und wenn mich auch der ganze Kreis verhoehnt; Mein Pathos braechte!!!!!!\n"
+					+ "dich gewiss zum Lachen, Haettst du dir nicht das Lachen abgewoehnt. Von!!!!!!\n"
+					+ "Sonn' und Welten weiss ich nichts zu sagen, Ich sehe nur, wie sich die!!!!!!\n"
+					+ "Menschen plagen. Der kleine Gott der Welt bleibt stets von gleichem!!!!!!\n"
+					+ "Schlag, Und ist so wunderlich als wie am ersten Tag. Ein wenig besser!!!!!!\n"
+					+ "wuerd er leben, Haettst du ihm nicht den Schein des Himmelslichts gegeben;!!!!!!\n"
+					+ "Er nennt's Vernunft und braucht's allein, Nur tierischer als jedes Tier!!!!!!\n"
+					+ "zu sein. Er scheint mir, mit Verlaub von euer Gnaden, Wie eine der!!!!!!\n"
+					+ "langbeinigen Zikaden, Die immer fliegt und fliegend springt Und gleich im!!!!!!\n"
+					+ "Gras ihr altes Liedchen singt; Und laeg er nur noch immer in dem Grase! In!!!!!!\n"
+					+ "jeden Quark begraebt er seine Nase.!!!!!!\n"
+					+ "DER HERR: Hast du mir weiter nichts zu sagen? Kommst du nur immer!!!!!!\n"
+					+ "anzuklagen? Ist auf der Erde ewig dir nichts recht?!!!!!!\n"
+					+ "MEPHISTOPHELES: Nein Herr! ich find es dort, wie immer, herzlich!!!!!!\n"
+					+ "schlecht. Die Menschen dauern mich in ihren Jammertagen, Ich mag sogar!!!!!!\n"
+					+ "die armen selbst nicht plagen.!!!!!!\n" + "DER HERR: Kennst du den Faust?!!!!!!\n"
+					+ "MEPHISTOPHELES: Den Doktor?!!!!!!\n"
+					+ "DER HERR: Meinen Knecht!!!!!!!\n"
+					+ "MEPHISTOPHELES: Fuerwahr! er dient Euch auf besondre Weise. Nicht irdisch!!!!!!\n"
+					+ "ist des Toren Trank noch Speise. Ihn treibt die Gaerung in die Ferne, Er!!!!!!\n"
+					+ "ist sich seiner Tollheit halb bewusst; Vom Himmel fordert er die schoensten!!!!!!\n"
+					+ "Sterne Und von der Erde jede hoechste Lust, Und alle Naeh und alle Ferne!!!!!!\n"
+					+ "Befriedigt nicht die tiefbewegte Brust.!!!!!!\n"
+					+ "DER HERR: Wenn er mir auch nur verworren dient, So werd ich ihn bald in!!!!!!\n"
+					+ "die Klarheit fuehren. Weiss doch der Gaertner, wenn das Baeumchen gruent, Das!!!!!!\n"
+					+ "Bluet und Frucht die kuenft'gen Jahre zieren.!!!!!!\n"
+					+ "MEPHISTOPHELES: Was wettet Ihr? den sollt Ihr noch verlieren! Wenn Ihr!!!!!!\n"
+					+ "mir die Erlaubnis gebt, Ihn meine Strasse sacht zu fuehren.!!!!!!\n"
+					+ "DER HERR: Solang er auf der Erde lebt, So lange sei dir's nicht verboten,!!!!!!\n"
+					+ "Es irrt der Mensch so lang er strebt.!!!!!!\n"
+					+ "MEPHISTOPHELES: Da dank ich Euch; denn mit den Toten Hab ich mich niemals!!!!!!\n"
+					+ "gern befangen. Am meisten lieb ich mir die vollen, frischen Wangen. Fuer!!!!!!\n"
+					+ "einem Leichnam bin ich nicht zu Haus; Mir geht es wie der Katze mit der Maus.!!!!!!\n"
+					+ "DER HERR: Nun gut, es sei dir ueberlassen! Zieh diesen Geist von seinem!!!!!!\n"
+					+ "Urquell ab, Und fuehr ihn, kannst du ihn erfassen, Auf deinem Wege mit!!!!!!\n"
+					+ "herab, Und steh beschaemt, wenn du bekennen musst: Ein guter Mensch, in!!!!!!\n"
+					+ "seinem dunklen Drange, Ist sich des rechten Weges wohl bewusst.!!!!!!\n"
+					+ "MEPHISTOPHELES: Schon gut! nur dauert es nicht lange. Mir ist fuer meine!!!!!!\n"
+					+ "Wette gar nicht bange. Wenn ich zu meinem Zweck gelange, Erlaubt Ihr mir!!!!!!\n"
+					+ "Triumph aus voller Brust. Staub soll er fressen, und mit Lust, Wie meine!!!!!!\n"
+					+ "Muhme, die beruehmte Schlange.!!!!!!\n"
+					+ "DER HERR: Du darfst auch da nur frei erscheinen; Ich habe deinesgleichen!!!!!!\n"
+					+ "nie gehasst. Von allen Geistern, die verneinen, ist mir der Schalk am!!!!!!\n"
+					+ "wenigsten zur Last. Des Menschen Taetigkeit kann allzu leicht erschlaffen,!!!!!!\n"
+					+ "er liebt sich bald die unbedingte Ruh; Drum geb ich gern ihm den Gesellen!!!!!!\n"
+					+ "zu, Der reizt und wirkt und muss als Teufel schaffen. Doch ihr, die echten!!!!!!\n"
+					+ "Goettersoehne, Erfreut euch der lebendig reichen Schoene! Das Werdende, das!!!!!!\n"
+					+ "ewig wirkt und lebt, Umfass euch mit der Liebe holden Schranken, Und was!!!!!!\n"
+					+ "in schwankender Erscheinung schwebt, Befestigt mit dauernden Gedanken!!!!!!!\n"
+					+ "(Der Himmel schliesst, die Erzengel verteilen sich.)!!!!!!\n"
+					+ "MEPHISTOPHELES (allein): Von Zeit zu Zeit seh ich den Alten gern, Und!!!!!!\n"
+					+ "huete mich, mit ihm zu brechen. Es ist gar huebsch von einem grossen Herrn,!!!!!!\n"
+					+ "So menschlich mit dem Teufel selbst zu sprechen.!!!!!!";
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/BoltSplitITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/BoltSplitITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/BoltSplitITCase.java
new file mode 100644
index 0000000..dc174e7
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/BoltSplitITCase.java
@@ -0,0 +1,28 @@
+/*
+ * 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.split;
+
+import org.junit.Test;
+
+public class BoltSplitITCase {
+
+	@Test
+	public void testTopology() throws Exception {
+		SplitStreamBoltLocal.main(new String[] { "0", "/dev/null" });
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java
new file mode 100644
index 0000000..c7b9c1d
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java
@@ -0,0 +1,61 @@
+/*
+ * 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.split;
+
+import java.util.Map;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+
+public class SplitBolt extends BaseRichBolt {
+	private static final long serialVersionUID = -6627606934204267173L;
+
+	public static final String EVEN_STREAM = "even";
+	public static final String ODD_STREAM = "odd";
+
+	private OutputCollector collector;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+		this.collector = collector;
+
+	}
+
+	@Override
+	public void execute(Tuple input) {
+		if (input.getInteger(0) % 2 == 0) {
+			this.collector.emit(EVEN_STREAM, new Values(input.getInteger(0)));
+		} else {
+			this.collector.emit(ODD_STREAM, new Values(input.getInteger(0)));
+		}
+	}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {
+		Fields schema = new Fields("number");
+		declarer.declareStream(EVEN_STREAM, schema);
+		declarer.declareStream(ODD_STREAM, schema);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java
new file mode 100644
index 0000000..d0973cb
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.storm.split;
+
+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";
+	public final static String boltId = "splitBolt";
+	public final static String evenVerifierId = "evenVerifier";
+	public final static String oddVerifierId = "oddVerifier";
+	public final static String sinkId = "sink";
+	private final static OutputFormatter formatter = new TupleOutputFormatter();
+
+	public static FlinkTopologyBuilder buildTopology() {
+		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
+
+		builder.setSpout(spoutId, new RandomSpout(false, seed));
+		builder.setBolt(boltId, new SplitBolt()).shuffleGrouping(spoutId);
+		builder.setBolt(evenVerifierId, new VerifyAndEnrichBolt(true)).shuffleGrouping(boltId,
+				SplitBolt.EVEN_STREAM);
+		builder.setBolt(oddVerifierId, new VerifyAndEnrichBolt(false)).shuffleGrouping(boltId,
+				SplitBolt.ODD_STREAM);
+
+		// emit result
+		if (outputPath != null) {
+			// read the text file from given input path
+			final String[] tokens = outputPath.split(":");
+			final String outputFile = tokens[tokens.length - 1];
+			builder.setBolt(sinkId, new BoltFileSink(outputFile, formatter))
+			.shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId);
+		} else {
+			builder.setBolt(sinkId, new BoltPrintSink(formatter), 4)
+			.shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId);
+		}
+
+		return builder;
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static long seed = System.currentTimeMillis();
+	private static String outputPath = null;
+
+	static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			if (args.length == 2) {
+				seed = Long.parseLong(args[0]);
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: SplitStreamBoltLocal <seed> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing SplitBoltTopology example with random data");
+			System.out.println("  Usage: SplitStreamBoltLocal <seed> <result path>");
+		}
+
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java
new file mode 100644
index 0000000..2b3b6a8
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java
@@ -0,0 +1,85 @@
+/*
+ * 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.split;
+
+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";
+	public final static String evenVerifierId = "evenVerifier";
+	public final static String oddVerifierId = "oddVerifier";
+	public final static String sinkId = "sink";
+	private final static OutputFormatter formatter = new TupleOutputFormatter();
+
+	public static FlinkTopologyBuilder buildTopology() {
+		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
+
+		builder.setSpout(spoutId, new RandomSpout(true, seed));
+		builder.setBolt(evenVerifierId, new VerifyAndEnrichBolt(true)).shuffleGrouping(spoutId,
+				RandomSpout.EVEN_STREAM);
+		builder.setBolt(oddVerifierId, new VerifyAndEnrichBolt(false)).shuffleGrouping(spoutId,
+				RandomSpout.ODD_STREAM);
+
+		// emit result
+		if (outputPath != null) {
+			// read the text file from given input path
+			final String[] tokens = outputPath.split(":");
+			final String outputFile = tokens[tokens.length - 1];
+			builder.setBolt(sinkId, new BoltFileSink(outputFile, formatter))
+			.shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId);
+		} else {
+			builder.setBolt(sinkId, new BoltPrintSink(formatter), 4)
+			.shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId);
+		}
+
+		return builder;
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static long seed = System.currentTimeMillis();
+	private static String outputPath = null;
+
+	static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			if (args.length == 2) {
+				seed = Long.parseLong(args[0]);
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: SplitStreamSpoutLocal <seed> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing SplitSpoutTopology example with random data");
+			System.out.println("  Usage: SplitStreamSpoutLocal <seed> <result path>");
+		}
+
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java
new file mode 100644
index 0000000..e2c22f9
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java
@@ -0,0 +1,51 @@
+/*
+ * 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.split;
+
+import org.apache.flink.storm.api.FlinkLocalCluster;
+import org.apache.flink.storm.api.FlinkTopologyBuilder;
+
+import backtype.storm.utils.Utils;
+
+public class SplitStreamBoltLocal {
+	public final static String topologyId = "Bolt split stream example";
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!SplitBoltTopology.parseParameters(args)) {
+			return;
+		}
+
+		// build Topology the Storm way
+		final FlinkTopologyBuilder builder = SplitBoltTopology.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();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java
new file mode 100644
index 0000000..2070f66
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java
@@ -0,0 +1,51 @@
+/*
+ * 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.split;
+
+import org.apache.flink.storm.api.FlinkLocalCluster;
+import org.apache.flink.storm.api.FlinkTopologyBuilder;
+
+import backtype.storm.utils.Utils;
+
+public class SplitStreamSpoutLocal {
+	public final static String topologyId = "Spout split stream example";
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!SplitSpoutTopology.parseParameters(args)) {
+			return;
+		}
+
+		// build Topology the Storm way
+		final FlinkTopologyBuilder builder = SplitSpoutTopology.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();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SpoutSplitITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SpoutSplitITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SpoutSplitITCase.java
new file mode 100644
index 0000000..8e0fda9
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SpoutSplitITCase.java
@@ -0,0 +1,28 @@
+/*
+ * 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.split;
+
+import org.junit.Test;
+
+public class SpoutSplitITCase {
+
+	@Test
+	public void testTopology() throws Exception {
+		SplitStreamSpoutLocal.main(new String[] { "0", "/dev/null" });
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/util/StormTestBase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/util/StormTestBase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/util/StormTestBase.java
new file mode 100644
index 0000000..32dac7b
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/util/StormTestBase.java
@@ -0,0 +1,117 @@
+/*
+ * 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.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;
+
+/**
+ * Base class for Storm tests.
+ */
+public abstract class StormTestBase extends AbstractTestBase {
+	
+	public static final int DEFAULT_PARALLELISM = 4;
+	
+	public StormTestBase() {
+		this(new Configuration());
+	}
+	
+	public StormTestBase(Configuration config) {
+		super(config, StreamingMode.STREAMING);
+		setTaskManagerNumSlots(DEFAULT_PARALLELISM);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Methods to create the test program and for pre- and post- test work
+	// ------------------------------------------------------------------------
+
+	protected abstract void testProgram() throws Exception;
+
+	protected void preSubmit() throws Exception {}
+
+	protected void postSubmit() throws Exception {}
+
+	// ------------------------------------------------------------------------
+	//  Test entry point
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testJob() throws Exception {
+		try {
+			// pre-submit
+			try {
+				preSubmit();
+			}
+			catch (Exception e) {
+				System.err.println(e.getMessage());
+				e.printStackTrace();
+				fail("Pre-submit work caused an error: " + e.getMessage());
+			}
+
+			// prepare the test environment
+			startCluster();
+
+			// we need to initialize the stream test environment, and the storm local cluster
+			TestStreamEnvironment.setAsContext(this.executor, DEFAULT_PARALLELISM);
+			
+			FlinkLocalCluster.initialize(new FlinkLocalCluster.LocalClusterFactory() {
+				@Override
+				public FlinkLocalCluster createLocalCluster() {
+					return new FlinkLocalCluster(executor);
+				}
+			});
+
+			// call the test program
+			try {
+				testProgram();
+			}
+			catch (Exception e) {
+				System.err.println(e.getMessage());
+				e.printStackTrace();
+				fail("Error while calling the test program: " + e.getMessage());
+			}
+
+			// post-submit
+			try {
+				postSubmit();
+			}
+			catch (Exception e) {
+				System.err.println(e.getMessage());
+				e.printStackTrace();
+				fail("Post-submit work caused an error: " + e.getMessage());
+			}
+		}
+		finally {
+			// reset the FlinkLocalCluster to its default behavior
+			FlinkLocalCluster.initialize(new FlinkLocalCluster.DefaultLocalClusterFactory());
+			
+			// reset the StreamExecutionEnvironment to its default behavior
+			TestStreamEnvironment.unsetAsContext();
+			
+			// clean up all resources
+			stopCluster();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountITCase.java
new file mode 100644
index 0000000..62d23ab
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountITCase.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.storm.wordcount;
+
+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 {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		BoltTokenizerWordCount.main(new String[]{this.textPath, this.resultPath});
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojoITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojoITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojoITCase.java
new file mode 100644
index 0000000..009bdc2
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojoITCase.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.storm.wordcount;
+
+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 {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		BoltTokenizerWordCountPojo.main(new String[]{this.textPath, this.resultPath});
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNamesITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNamesITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNamesITCase.java
new file mode 100644
index 0000000..321015b
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNamesITCase.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.storm.wordcount;
+
+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 {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		BoltTokenizerWordCountWithNames.main(new String[]{this.textPath, this.resultPath});
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/SpoutSourceWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/SpoutSourceWordCountITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/SpoutSourceWordCountITCase.java
new file mode 100644
index 0000000..0cff211
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/wordcount/SpoutSourceWordCountITCase.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.storm.wordcount;
+
+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 {
+
+	protected String textPath;
+	protected String resultPath;
+
+	@Override
+	protected void preSubmit() throws Exception {
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+
+	@Override
+	protected void postSubmit() throws Exception {
+		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
+	}
+
+	@Override
+	protected void testProgram() throws Exception {
+		SpoutSourceWordCount.main(new String[]{this.textPath, this.resultPath});
+	}
+
+}


[11/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapper.java
----------------------------------------------------------------------
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 300b241..0000000
--- 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<OUT> extends AbstractStormSpoutWrapper<OUT> {
-	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<String> 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();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormTuple.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormTuple.java
deleted file mode 100644
index 07d94b4..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormTuple.java
+++ /dev/null
@@ -1,327 +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;
-
-/*
- * We do neither import
- * 		backtype.storm.tuple.Tuple;
- * nor
- * 		org.apache.flink.api.java.tuple.Tuple
- * to avoid confusion
- */
-
-import backtype.storm.generated.GlobalStreamId;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.MessageId;
-import backtype.storm.tuple.Values;
-
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.util.List;
-
-/**
- * {@link StormTuple} converts a Flink tuple of type {@code IN} into a Storm tuple.
- */
-class StormTuple<IN> implements backtype.storm.tuple.Tuple {
-
-	/** The Storm representation of the original Flink tuple */
-	private final Values stormTuple;
-	/** The schema (ie, ordered field names) of the tuple */
-	private final Fields schema;
-
-	/**
-	 * Create a new Storm tuple from the given Flink tuple. The provided {@code nameIndexMap} is ignored for raw input
-	 * types.
-	 * 
-	 * @param flinkTuple
-	 * 		The Flink tuple to be converted.
-	 * @param schema
-	 * 		The schema (ie, ordered field names) of the tuple.
-	 */
-	public 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;
-
-			final int numberOfAttributes = t.getArity();
-			this.stormTuple = new Values();
-			for (int i = 0; i < numberOfAttributes; ++i) {
-				this.stormTuple.add(t.getField(i));
-			}
-		} else {
-			this.schema = null;
-			this.stormTuple = new Values(flinkTuple);
-		}
-	}
-
-	@Override
-	public int size() {
-		return this.stormTuple.size();
-	}
-
-	@Override
-	public boolean contains(final String field) {
-		if (this.schema != null) {
-			return this.schema.contains(field);
-		}
-
-		try {
-			this.getPublicMemberField(field);
-			return true;
-		} catch (NoSuchFieldException f) {
-			try {
-				this.getGetterMethod(field);
-				return true;
-			} catch (Exception g) {
-				return false;
-			}
-		} catch (Exception e) {
-			return false;
-		}
-	}
-
-	@Override
-	public Fields getFields() {
-		return this.schema;
-	}
-
-	@Override
-	public int fieldIndex(final String field) {
-		return this.schema.fieldIndex(field);
-	}
-
-	@Override
-	public List<Object> select(final Fields selector) {
-		return this.schema.select(selector, this.stormTuple);
-	}
-
-	@Override
-	public Object getValue(final int i) {
-		return this.stormTuple.get(i);
-	}
-
-	@Override
-	public String getString(final int i) {
-		return (String) this.stormTuple.get(i);
-	}
-
-	@Override
-	public Integer getInteger(final int i) {
-		return (Integer) this.stormTuple.get(i);
-	}
-
-	@Override
-	public Long getLong(final int i) {
-		return (Long) this.stormTuple.get(i);
-	}
-
-	@Override
-	public Boolean getBoolean(final int i) {
-		return (Boolean) this.stormTuple.get(i);
-	}
-
-	@Override
-	public Short getShort(final int i) {
-		return (Short) this.stormTuple.get(i);
-	}
-
-	@Override
-	public Byte getByte(final int i) {
-		return (Byte) this.stormTuple.get(i);
-	}
-
-	@Override
-	public Double getDouble(final int i) {
-		return (Double) this.stormTuple.get(i);
-	}
-
-	@Override
-	public Float getFloat(final int i) {
-		return (Float) this.stormTuple.get(i);
-	}
-
-	@Override
-	public byte[] getBinary(final int i) {
-		return (byte[]) this.stormTuple.get(i);
-	}
-
-	private Field getPublicMemberField(final String field) throws Exception {
-		assert (this.stormTuple.size() == 1);
-		return this.stormTuple.get(0).getClass().getField(field);
-	}
-
-	private Method getGetterMethod(final String field) throws Exception {
-		assert (this.stormTuple.size() == 1);
-		return this.stormTuple
-				.get(0)
-				.getClass()
-				.getMethod("get" + Character.toUpperCase(field.charAt(0)) + field.substring(1),
-						(Class<?>[]) null);
-	}
-
-	private Object getValueByPublicMember(final String field) throws Exception {
-		assert (this.stormTuple.size() == 1);
-		return getPublicMemberField(field).get(this.stormTuple.get(0));
-	}
-
-	private Object getValueByGetter(final String field) throws Exception {
-		assert (this.stormTuple.size() == 1);
-		return getGetterMethod(field).invoke(this.stormTuple.get(0), (Object[]) null);
-	}
-
-	@SuppressWarnings("unchecked")
-	public <T> T getValueByName(final String field) {
-		if (this.schema != null) {
-			return (T) this.getValue(this.schema.fieldIndex(field));
-		}
-		assert (this.stormTuple.size() == 1);
-
-		Exception e;
-		try {
-			// try public member
-			return (T) getValueByPublicMember(field);
-		} catch (NoSuchFieldException f) {
-			try {
-				// try getter-method
-				return (T) getValueByGetter(field);
-			} catch (Exception g) {
-				e = g;
-			}
-		} catch (Exception f) {
-			e = f;
-		}
-
-		throw new RuntimeException("Could not access field <" + field + ">", e);
-	}
-
-	@Override
-	public Object getValueByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public String getStringByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public Integer getIntegerByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public Long getLongByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public Boolean getBooleanByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public Short getShortByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public Byte getByteByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public Double getDoubleByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public Float getFloatByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public byte[] getBinaryByField(final String field) {
-		return getValueByName(field);
-	}
-
-	@Override
-	public List<Object> getValues() {
-		return this.stormTuple;
-	}
-
-	@Override
-	public GlobalStreamId getSourceGlobalStreamid() {
-		// not sure if Flink can support this
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public String getSourceComponent() {
-		// not sure if Flink can support this
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public int getSourceTask() {
-		// not sure if Flink can support this
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public String getSourceStreamId() {
-		// not sure if Flink can support this
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public MessageId getMessageId() {
-		// not sure if Flink can support this
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public int hashCode() {
-		final int prime = 31;
-		int result = 1;
-		result = (prime * result) + ((this.stormTuple == null) ? 0 : this.stormTuple.hashCode());
-		return result;
-	}
-
-	@Override
-	public boolean equals(final Object obj) {
-		if (this == obj) {
-			return true;
-		}
-		if (obj == null) {
-			return false;
-		}
-		if (this.getClass() != obj.getClass()) {
-			return false;
-		}
-		final StormTuple<?> other = (StormTuple<?>) obj;
-		if (this.stormTuple == null) {
-			if (other.stormTuple != null) {
-				return false;
-			}
-		} else if (!this.stormTuple.equals(other.stormTuple)) {
-			return false;
-		}
-		return true;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 891497e..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelper.java
+++ /dev/null
@@ -1,268 +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.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.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} 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
-	 * output type {@link org.apache.flink.api.java.tuple.Tuple0 Tuple0} to
-	 * {@link org.apache.flink.api.java.tuple.Tuple25 Tuple25}.
-	 * 
-	 * @param spoutOrBolt
-	 *            The Storm {@link IRichSpout spout} or {@link IRichBolt bolt} to be used.
-	 * @param rawOutputs
-	 *            Contains stream names if a single attribute output stream, should not be of type
-	 *            {@link org.apache.flink.api.java.tuple.Tuple1 Tuple1} but be of a raw type. (Can be {@code null}.)
-	 * @return The number of attributes to be used for each stream.
-	 * @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 static HashMap<String, Integer> getNumberOfAttributes(final IComponent spoutOrBolt,
-			final Collection<String> rawOutputs)
-					throws IllegalArgumentException {
-		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-		spoutOrBolt.declareOutputFields(declarer);
-
-		for (Entry<String, Integer> schema : declarer.outputSchemas.entrySet()) {
-			int declaredNumberOfAttributes = schema.getValue();
-			if ((declaredNumberOfAttributes < 0) || (declaredNumberOfAttributes > 25)) {
-				throw new IllegalArgumentException(
-						"Provided bolt declares non supported number of output attributes. Must be in range [0;25] but "
-								+ "was " + declaredNumberOfAttributes);
-			}
-
-			if (rawOutputs != null && rawOutputs.contains(schema.getKey())) {
-				if (declaredNumberOfAttributes != 1) {
-					throw new IllegalArgumentException(
-							"Ouput type is requested to be raw type, but provided bolt declares more then one output "
-									+ "attribute.");
-				}
-				schema.setValue(-1);
-			}
-		}
-
-		return declarer.outputSchemas;
-	}
-
-	/** 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<Integer, String> taskToComponents = new HashMap<Integer, String>();
-		final Map<String, List<Integer>> componentToSortedTasks = new HashMap<String, List<Integer>>();
-		final Map<String, Map<String, Fields>> componentToStreamToFields = new HashMap<String, Map<String, Fields>>();
-		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<Integer> workerTasks = new ArrayList<Integer>();
-		final Map<String, Object> defaultResources = new HashMap<String, Object>();
-		final Map<String, Object> userResources = new HashMap<String, Object>();
-		final Map<String, Object> executorData = new HashMap<String, Object>();
-		final Map registeredMetrics = new HashMap();
-		Atom openOrPrepareWasCalled = null;
-
-		if (stormTopology == null) {
-			// embedded mode
-			ComponentCommon common = new ComponentCommon();
-			common.set_parallelism_hint(dop);
-
-			HashMap<String, SpoutSpec> spouts = new HashMap<String, SpoutSpec>();
-			HashMap<String, Bolt> bolts = new HashMap<String, Bolt>();
-			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<String, StateSpoutSpec>());
-
-			taskId = context.getIndexOfThisSubtask();
-
-			List<Integer> sortedTasks = new ArrayList<Integer>(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 {
-			// whole topology is built (ie, FlinkTopologyBuilder is used)
-			Map<String, SpoutSpec> spouts = stormTopology.get_spouts();
-			Map<String, Bolt> bolts = stormTopology.get_bolts();
-			Map<String, StateSpoutSpec> stateSpouts = stormTopology.get_state_spouts();
-
-			tid = 1;
-
-			for (Entry<String, SpoutSpec> 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<String, Bolt> 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<String, StateSpoutSpec> 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<Integer, String> taskToComponents,
-			final Map<String, List<Integer>> componentToSortedTasks,
-			final Map<String, Map<String, Fields>> componentToStreamToFields) {
-		final int parallelism_hint = common.get_parallelism_hint();
-		Integer taskId = null;
-
-		if (componentId.equals(operatorName)) {
-			taskId = tid + index;
-		}
-
-		List<Integer> sortedTasks = new ArrayList<Integer>(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<String, Fields> outputStreams = new HashMap<String, Fields>();
-		for(Entry<String, StreamInfo> outStream : common.get_streams().entrySet()) {
-			outputStreams.put(outStream.getKey(), new Fields(outStream.getValue().get_output_fields()));
-		}
-		componentToStreamToFields.put(componentId, outputStreams);
-
-		return taskId;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.java
deleted file mode 100644
index 0dd9b1c..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyBuilderTest.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.api;
-
-import org.apache.flink.stormcompatibility.util.TestDummyBolt;
-import org.apache.flink.stormcompatibility.util.TestDummySpout;
-import org.apache.flink.stormcompatibility.util.TestSink;
-import org.junit.Test;
-
-import backtype.storm.tuple.Fields;
-
-public class FlinkTopologyBuilderTest {
-
-	@Test(expected = RuntimeException.class)
-	public void testUnknowSpout() {
-		FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
-		builder.setSpout("spout", new TestSpout());
-		builder.setBolt("bolt", new TestBolt()).shuffleGrouping("unknown");
-		builder.createTopology();
-	}
-
-	@Test(expected = RuntimeException.class)
-	public void testUnknowBolt() {
-		FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
-		builder.setSpout("spout", new TestSpout());
-		builder.setBolt("bolt1", new TestBolt()).shuffleGrouping("spout");
-		builder.setBolt("bolt2", new TestBolt()).shuffleGrouping("unknown");
-		builder.createTopology();
-	}
-
-	@Test(expected = RuntimeException.class)
-	public void testUndeclaredStream() {
-		FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
-		builder.setSpout("spout", new TestSpout());
-		builder.setBolt("bolt", new TestBolt()).shuffleGrouping("spout");
-		builder.createTopology();
-	}
-
-	@Test
-	public void testFieldsGroupingOnMultipleSpoutOutputStreams() {
-		FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
-
-		flinkBuilder.setSpout("spout", new TestDummySpout());
-		flinkBuilder.setBolt("sink", new TestSink()).fieldsGrouping("spout",
-				TestDummySpout.spoutStreamId, new Fields("id"));
-
-		flinkBuilder.createTopology();
-	}
-
-	@Test
-	public void testFieldsGroupingOnMultipleBoltOutputStreams() {
-		FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
-
-		flinkBuilder.setSpout("spout", new TestDummySpout());
-		flinkBuilder.setBolt("bolt", new TestDummyBolt()).shuffleGrouping("spout");
-		flinkBuilder.setBolt("sink", new TestSink()).fieldsGrouping("bolt",
-				TestDummyBolt.groupingStreamId, new Fields("id"));
-
-		flinkBuilder.createTopology();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java
----------------------------------------------------------------------
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
deleted file mode 100644
index c98c9a3..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTopologyTest.java
+++ /dev/null
@@ -1,62 +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.junit.Assert;
-import org.junit.Test;
-
-public class FlinkTopologyTest {
-
-	@Test
-	public void testDefaultParallelism() {
-		final FlinkTopology topology = new FlinkTopology();
-		Assert.assertEquals(1, topology.getParallelism());
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testExecute() throws Exception {
-		new FlinkTopology().execute();
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testExecuteWithName() throws Exception {
-		new FlinkTopology().execute(null);
-	}
-
-	@Test
-	public void testNumberOfTasks() {
-		final FlinkTopology topology = new FlinkTopology();
-
-		Assert.assertEquals(0, topology.getNumberOfTasks());
-
-		topology.increaseNumberOfTasks(3);
-		Assert.assertEquals(3, topology.getNumberOfTasks());
-
-		topology.increaseNumberOfTasks(2);
-		Assert.assertEquals(5, topology.getNumberOfTasks());
-
-		topology.increaseNumberOfTasks(8);
-		Assert.assertEquals(13, topology.getNumberOfTasks());
-	}
-
-	@Test(expected = AssertionError.class)
-	public void testAssert() {
-		new FlinkTopology().increaseNumberOfTasks(0);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java
deleted file mode 100644
index 2e4a534..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestBolt.java
+++ /dev/null
@@ -1,48 +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 java.util.Map;
-
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IRichBolt;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Tuple;
-
-public class TestBolt implements IRichBolt {
-	private static final long serialVersionUID = -667148827441397683L;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {}
-
-	@Override
-	public void execute(Tuple input) {}
-
-	@Override
-	public void cleanup() {}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java
deleted file mode 100644
index 146218f..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestSpout.java
+++ /dev/null
@@ -1,59 +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 java.util.Map;
-
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IRichSpout;
-import backtype.storm.topology.OutputFieldsDeclarer;
-
-public class TestSpout implements IRichSpout {
-	private static final long serialVersionUID = -4884029383198924007L;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {}
-
-	@Override
-	public void close() {}
-
-	@Override
-	public void activate() {}
-
-	@Override
-	public void deactivate() {}
-
-	@Override
-	public void nextTuple() {}
-
-	@Override
-	public void ack(Object msgId) {}
-
-	@Override
-	public void fail(Object msgId) {}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java
----------------------------------------------------------------------
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
deleted file mode 100644
index f664e58..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/api/TestTopologyBuilder.java
+++ /dev/null
@@ -1,27 +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.generated.StormTopology;
-
-public class TestTopologyBuilder extends FlinkTopologyBuilder {
-	@Override
-	public StormTopology getStormTopology() {
-		return super.getStormTopology();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/AbstractTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/AbstractTest.java
deleted file mode 100644
index 94a50cf..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/AbstractTest.java
+++ /dev/null
@@ -1,39 +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.util;
-
-import org.junit.Before;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Random;
-
-public abstract class AbstractTest {
-	private static final Logger LOG = LoggerFactory.getLogger(AbstractTest.class);
-
-	protected long seed;
-	protected Random r;
-
-	@Before
-	public void prepare() {
-		this.seed = System.currentTimeMillis();
-		this.r = new Random(this.seed);
-		LOG.info("Test seed: {}", new Long(this.seed));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java
deleted file mode 100644
index 1891873..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java
+++ /dev/null
@@ -1,77 +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.util;
-
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IRichSpout;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
-
-import java.util.Map;
-
-public class FiniteTestSpout implements IRichSpout {
-	private static final long serialVersionUID = 7992419478267824279L;
-
-	private int numberOfOutputTuples;
-	private SpoutOutputCollector collector;
-
-	public FiniteTestSpout(final int numberOfOutputTuples) {
-		this.numberOfOutputTuples = numberOfOutputTuples;
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void close() {/* nothing to do */}
-
-	@Override
-	public void activate() {/* nothing to do */}
-
-	@Override
-	public void deactivate() {/* nothing to do */}
-
-	@Override
-	public void nextTuple() {
-		if (--this.numberOfOutputTuples >= 0) {
-			this.collector.emit(new Values(new Integer(this.numberOfOutputTuples)));
-		}
-	}
-
-	@Override
-	public void ack(final Object msgId) {/* nothing to do */}
-
-	@Override
-	public void fail(final Object msgId) {/* nothing to do */}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields("dummy"));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java
deleted file mode 100644
index 8e63563..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java
+++ /dev/null
@@ -1,193 +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.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.FlinkOutputFieldsDeclarer;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.LinkedList;
-
-public class FlinkOutputFieldsDeclarerTest extends AbstractTest {
-
-
-
-	@Test
-	public void testNull() {
-		Assert.assertNull(new FlinkOutputFieldsDeclarer().getOutputType(null));
-	}
-
-	@Test
-	public void testDeclare() {
-		for (int i = 0; i < 2; ++i) { // test case: simple / non-direct
-			for (int j = 1; j < 2; ++j) { // number of streams
-				for (int k = 0; k <= 25; ++k) { // number of attributes
-					this.runDeclareTest(i, j, k);
-				}
-			}
-		}
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareSimpleToManyAttributes() {
-		this.runDeclareTest(0, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareNonDirectToManyAttributes() {
-		this.runDeclareTest(1, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareDefaultStreamToManyAttributes() {
-		this.runDeclareTest(2, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareFullToManyAttributes() {
-		this.runDeclareTest(3, this.r.nextBoolean() ? 1 : 2, 26);
-	}
-
-	private void runDeclareTest(final int testCase, final int numberOfStreams,
-			final int numberOfAttributes) {
-		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-
-		String[] streams = null;
-		if (numberOfStreams > 1 || r.nextBoolean()) {
-			streams = new String[numberOfStreams];
-			for (int i = 0; i < numberOfStreams; ++i) {
-				streams[i] = "stream" + i;
-			}
-		}
-
-		final String[] attributes = new String[numberOfAttributes];
-		for (int i = 0; i < attributes.length; ++i) {
-			attributes[i] = "a" + i;
-		}
-
-		switch (testCase) {
-		case 0:
-			this.declareSimple(declarer, streams, attributes);
-			break;
-		default:
-			this.declareNonDirect(declarer, streams, attributes);
-		}
-
-		if (streams == null) {
-			streams = new String[] { Utils.DEFAULT_STREAM_ID };
-		}
-
-		for (String stream : streams) {
-			final TypeInformation<?> type = declarer.getOutputType(stream);
-
-			if (numberOfAttributes == 1) {
-				Assert.assertEquals(type.getClass(), GenericTypeInfo.class);
-				Assert.assertEquals(type.getTypeClass(), Object.class);
-			} else {
-				Assert.assertEquals(numberOfAttributes, type.getArity());
-				Assert.assertTrue(type.isTupleType());
-			}
-		}
-	}
-
-	private void declareSimple(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
-			final String[] attributes) {
-
-		if (streams != null) {
-			for (String stream : streams) {
-				declarer.declareStream(stream, new Fields(attributes));
-			}
-		} else {
-			declarer.declare(new Fields(attributes));
-		}
-	}
-
-	private void declareNonDirect(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
-			final String[] attributes) {
-
-		if (streams != null) {
-			for (String stream : streams) {
-				declarer.declareStream(stream, false, new Fields(attributes));
-			}
-		} else {
-			declarer.declare(false, new Fields(attributes));
-		}
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testUndeclared() {
-		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-		declarer.getOutputType("unknownStreamId");
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareDirect() {
-		new FlinkOutputFieldsDeclarer().declare(true, null);
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareDirect2() {
-		new FlinkOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, null);
-	}
-
-	@Test
-	public void testGetGroupingFieldIndexes() {
-		final int numberOfAttributes = 5 + this.r.nextInt(21);
-		final String[] attributes = new String[numberOfAttributes];
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			attributes[i] = "a" + i;
-		}
-
-		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
-		declarer.declare(new Fields(attributes));
-
-		final int numberOfKeys = 1 + this.r.nextInt(25);
-		final LinkedList<String> groupingFields = new LinkedList<String>();
-		final boolean[] indexes = new boolean[numberOfAttributes];
-
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			if (this.r.nextInt(26) < numberOfKeys) {
-				groupingFields.add(attributes[i]);
-				indexes[i] = true;
-			} else {
-				indexes[i] = false;
-			}
-		}
-
-		final int[] expectedResult = new int[groupingFields.size()];
-		int j = 0;
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			if (indexes[i]) {
-				expectedResult[j++] = i;
-			}
-		}
-
-		final int[] result = declarer.getGroupingFieldIndexes(Utils.DEFAULT_STREAM_ID,
-				groupingFields);
-
-		Assert.assertEquals(expectedResult.length, result.length);
-		for (int i = 0; i < expectedResult.length; ++i) {
-			Assert.assertEquals(expectedResult[i], result[i]);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java
deleted file mode 100644
index c3cb7d7..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java
+++ /dev/null
@@ -1,51 +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.util;
-
-import java.util.Iterator;
-
-import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector;
-import org.apache.flink.stormcompatibility.util.SplitStreamType;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class FlinkStormStreamSelectorTest {
-
-	@Test
-	public void testSelector() {
-		FlinkStormStreamSelector<Object> selector = new FlinkStormStreamSelector<Object>();
-		SplitStreamType<Object> tuple = new SplitStreamType<Object>();
-		Iterator<String> result;
-
-		tuple.streamId = "stream1";
-		result = selector.select(tuple).iterator();
-		Assert.assertEquals("stream1", result.next());
-		Assert.assertFalse(result.hasNext());
-
-		tuple.streamId = "stream2";
-		result = selector.select(tuple).iterator();
-		Assert.assertEquals("stream2", result.next());
-		Assert.assertFalse(result.hasNext());
-
-		tuple.streamId = "stream1";
-		result = selector.select(tuple).iterator();
-		Assert.assertEquals("stream1", result.next());
-		Assert.assertFalse(result.hasNext());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java
----------------------------------------------------------------------
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
deleted file mode 100644
index bd9ea3f..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java
+++ /dev/null
@@ -1,114 +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.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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
-				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), null, null,
-				null, null, null, null, null, null, null, null, null, null, null, null, null)
-		.setSubscribedState(null, null, null);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java
----------------------------------------------------------------------
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
deleted file mode 100644
index b499373..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java
+++ /dev/null
@@ -1,71 +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.util;
-
-import java.util.Map;
-
-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.tuple.Values;
-
-public class TestDummyBolt implements IRichBolt {
-	private static final long serialVersionUID = 6893611247443121322L;
-
-	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) {
-		this.context = context;
-		this.collector = collector;
-	}
-
-	@Override
-	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() {}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {
-		declarer.declareStream(shuffleStreamId, new Fields("data"));
-		declarer.declareStream(groupingStreamId, new Fields("id", "data"));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 345ca12..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java
+++ /dev/null
@@ -1,79 +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.util;
-
-import java.util.Map;
-
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-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 {
-	private static final long serialVersionUID = -5190945609124603118L;
-
-	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) {
-		this.context = context;
-		this.collector = collector;
-	}
-
-	@Override
-	public void close() {}
-
-	@Override
-	public void activate() {}
-
-	@Override
-	public void deactivate() {}
-
-	@Override
-	public void nextTuple() {
-		if (this.emit) {
-			this.collector.emit(new Values(this.context));
-			this.emit = false;
-		}
-	}
-
-	@Override
-	public void ack(Object msgId) {}
-
-	@Override
-	public void fail(Object msgId) {}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {
-		declarer.declareStream(Utils.DEFAULT_STREAM_ID, new Fields("data"));
-		declarer.declareStream(spoutStreamId, new Fields("id", "data"));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java
----------------------------------------------------------------------
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
deleted file mode 100644
index c8e5584..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java
+++ /dev/null
@@ -1,60 +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.util;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IRichBolt;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Tuple;
-
-public class TestSink implements IRichBolt {
-	private static final long serialVersionUID = 4314871456719370877L;
-
-	public final static List<TopologyContext> result = new LinkedList<TopologyContext>();
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-		result.add(context);
-	}
-
-	@Override
-	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() {}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java
----------------------------------------------------------------------
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 b44e8a1..0000000
--- 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<Object>(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<Object>(stormSpout);
-		wrapper.setRuntimeContext(taskContext);
-
-		wrapper.run(mock(SourceContext.class));
-		verify(stormSpout, never()).nextTuple();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java
deleted file mode 100644
index 738eb1e..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java
+++ /dev/null
@@ -1,91 +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.tuple.Fields;
-import backtype.storm.utils.Utils;
-
-import org.apache.flink.stormcompatibility.util.AbstractTest;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-
-public class SetupOutputFieldsDeclarerTest extends AbstractTest {
-
-	@Test
-	public void testDeclare() {
-		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
-
-		int numberOfAttributes = this.r.nextInt(26);
-		declarer.declare(createSchema(numberOfAttributes));
-		Assert.assertEquals(1, declarer.outputSchemas.size());
-		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(Utils.DEFAULT_STREAM_ID)
-				.intValue());
-
-		final String sid = "streamId";
-		numberOfAttributes = this.r.nextInt(26);
-		declarer.declareStream(sid, createSchema(numberOfAttributes));
-		Assert.assertEquals(2, declarer.outputSchemas.size());
-		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(sid).intValue());
-	}
-
-	private Fields createSchema(final int numberOfAttributes) {
-		final ArrayList<String> schema = new ArrayList<String>(numberOfAttributes);
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			schema.add("a" + i);
-		}
-		return new Fields(schema);
-	}
-
-	@Test
-	public void testDeclareDirect() {
-		new SetupOutputFieldsDeclarer().declare(false, new Fields());
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareDirectFail() {
-		new SetupOutputFieldsDeclarer().declare(true, new Fields());
-	}
-
-	@Test
-	public void testDeclareStream() {
-		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, new Fields());
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareStreamFail() {
-		new SetupOutputFieldsDeclarer().declareStream(null, new Fields());
-	}
-
-	@Test
-	public void testDeclareFullStream() {
-		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields());
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareFullStreamFailNonDefaultStream() {
-		new SetupOutputFieldsDeclarer().declareStream(null, false, new Fields());
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareFullStreamFailDirect() {
-		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, new Fields());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.java
deleted file mode 100644
index d01c3e0..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollectorTest.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 backtype.storm.tuple.Values;
-
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.stormcompatibility.util.AbstractTest;
-import org.apache.flink.streaming.api.operators.Output;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-
-public class StormBoltCollectorTest extends AbstractTest {
-
-	@SuppressWarnings({ "rawtypes", "unchecked" })
-	@Test
-	public void testBoltStormCollector() throws InstantiationException, IllegalAccessException {
-		for (int numberOfAttributes = -1; numberOfAttributes < 26; ++numberOfAttributes) {
-			final Output flinkCollector = mock(Output.class);
-			Tuple flinkTuple = null;
-			final Values tuple = new Values();
-
-			StormBoltCollector<?> collector;
-
-			final String streamId = "streamId";
-			HashMap<String, Integer> attributes = new HashMap<String, Integer>();
-			attributes.put(streamId, numberOfAttributes);
-
-			if (numberOfAttributes == -1) {
-				collector = new StormBoltCollector(attributes, flinkCollector);
-				tuple.add(new Integer(this.r.nextInt()));
-
-			} else {
-				collector = new StormBoltCollector(attributes, flinkCollector);
-				flinkTuple = Tuple.getTupleClass(numberOfAttributes).newInstance();
-
-				for (int i = 0; i < numberOfAttributes; ++i) {
-					tuple.add(new Integer(this.r.nextInt()));
-					flinkTuple.setField(tuple.get(i), i);
-				}
-			}
-
-			final Collection anchors = mock(Collection.class);
-			final List<Integer> taskIds;
-			taskIds = collector.emit(streamId, anchors, tuple);
-
-			Assert.assertNull(taskIds);
-
-			if (numberOfAttributes == -1) {
-				verify(flinkCollector).collect(tuple.get(0));
-			} else {
-				verify(flinkCollector).collect(flinkTuple);
-			}
-		}
-	}
-
-
-	@SuppressWarnings("unchecked")
-	@Test(expected = UnsupportedOperationException.class)
-	public void testReportError() {
-		new StormBoltCollector<Object>(mock(HashMap.class), mock(Output.class)).reportError(null);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test(expected = UnsupportedOperationException.class)
-	public void testEmitDirect() {
-		new StormBoltCollector<Object>(mock(HashMap.class), mock(Output.class)).emitDirect(0, null,
-				null, null);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test(expected = UnsupportedOperationException.class)
-	public void testAck() {
-		new StormBoltCollector<Object>(mock(HashMap.class), mock(Output.class)).ack(null);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Test(expected = UnsupportedOperationException.class)
-	public void testFail() {
-		new StormBoltCollector<Object>(mock(HashMap.class), mock(Output.class)).fail(null);
-	}
-
-}


[09/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml
deleted file mode 100644
index 2f3c02d..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/pom.xml
+++ /dev/null
@@ -1,362 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-storm-compatibility-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-storm-compatibility-examples</artifactId>
-	<name>flink-storm-compatibility-examples</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-storm-compatibility-core</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-java-examples</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-core</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- get default data from flink-java-examples package -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-dependency-plugin</artifactId>
-				<version>2.9</version><!--$NO-MVN-MAN-VER$-->
-				<executions>
-					<execution>
-						<id>unpack</id>
-						<phase>prepare-package</phase>
-						<goals>
-							<goal>unpack</goal>
-						</goals>
-						<configuration>
-							<artifactItems>
-								<artifactItem>
-									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-java-examples</artifactId>
-									<version>${project.version}</version>
-									<type>jar</type>
-									<overWrite>false</overWrite>
-									<outputDirectory>${project.build.directory}/classes</outputDirectory>
-									<includes>org/apache/flink/examples/java/wordcount/util/WordCountData.class</includes>
-								</artifactItem>
-								<artifactItem>
-									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-storm-compatibility-core</artifactId>
-									<version>${project.version}</version>
-									<type>jar</type>
-									<overWrite>false</overWrite>
-									<outputDirectory>${project.build.directory}/classes</outputDirectory>
-								</artifactItem>
-								<artifactItem>
-									<groupId>org.apache.storm</groupId>
-									<artifactId>storm-core</artifactId>
-									<version>0.9.4</version>
-									<type>jar</type>
-									<overWrite>false</overWrite>
-									<outputDirectory>${project.build.directory}/classes</outputDirectory>
-									<!-- need to exclude to be able to run
-									       * StormWordCountRemoteByClient and
-									       * StormWordCountRemoteBySubmitter
-									     within Eclipse -->
-									<excludes>defaults.yaml</excludes>
-								</artifactItem>
-								<artifactItem>
-									<groupId>com.googlecode.json-simple</groupId>
-									<artifactId>json-simple</artifactId>
-									<version>1.1</version>
-									<type>jar</type>
-									<overWrite>false</overWrite>
-									<outputDirectory>${project.build.directory}/classes</outputDirectory>
-								</artifactItem>
-								<artifactItem>
-									<groupId>org.yaml</groupId>
-									<artifactId>snakeyaml</artifactId>
-									<version>1.11</version>
-									<type>jar</type>
-									<overWrite>false</overWrite>
-									<outputDirectory>${project.build.directory}/classes</outputDirectory>
-								</artifactItem>
-							</artifactItems>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- self-contained jars for each example -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-
-				<executions>
-
-					<!-- WordCount Spout source-->
-					<!-- example for embedded spout - for whole topologies see "WordCount Storm topology" example below -->
-					<execution>
-						<id>WordCount-SpoutSource</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<finalName>WordCount</finalName>
-							<classifier>SpoutSource</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.stormcompatibility.wordcount.SpoutSourceWordCount</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<!-- from storm-core -->
-								<include>backtype/storm/topology/*.class</include>
-								<include>backtype/storm/spout/*.class</include>
-								<include>backtype/storm/task/*.class</include>
-								<include>backtype/storm/tuple/*.class</include>
-								<include>backtype/storm/generated/*.class</include>
-								<include>backtype/storm/metric/**/*.class</include>
-								<include>org/apache/thrift7/**/*.class</include>
-								<!-- Storm's recursive dependencies -->
-								<include>org/json/simple/**/*.class</include>
-								<!-- compatibility layer -->
-								<include>org/apache/flink/stormcompatibility/api/*.class</include>
-								<include>org/apache/flink/stormcompatibility/wrappers/*.class</include>
-								<!-- Word Count -->
-								<include>org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.class</include>
-								<include>org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount$*.class</include>
-								<include>org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.class</include>
-								<include>org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.class</include>
-								<include>org/apache/flink/stormcompatibility/util/AbstractStormSpout.class</include>
-								<include>org/apache/flink/stormcompatibility/util/StormFileSpout.class</include>
-								<include>org/apache/flink/stormcompatibility/util/StormInMemorySpout.class</include>
-								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- WordCount Bolt tokenizer-->
-					<!-- example for embedded bolt - for whole topologies see "WordCount Storm topology" example below -->
-					<execution>
-						<id>WordCount-BoltTokenizer</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<finalName>WordCount</finalName>
-							<classifier>BoltTokenizer</classifier>
-
-							<archive>
-								<manifestEntries>
-									<program-class>org.apache.flink.stormcompatibility.wordcount.BoltTokenizerWordCount</program-class>
-								</manifestEntries>
-							</archive>
-
-							<includes>
-								<!-- from storm-core -->
-								<include>backtype/storm/topology/*.class</include>
-								<include>backtype/storm/spout/*.class</include>
-								<include>backtype/storm/task/*.class</include>
-								<include>backtype/storm/tuple/*.class</include>
-								<include>backtype/storm/generated/*.class</include>
-								<include>backtype/storm/metric/**/*.class</include>
-								<include>org/apache/thrift7/**/*.class</include>
-								<!-- Storm's recursive dependencies -->
-								<include>org/json/simple/**/*.class</include>
-								<!-- compatibility layer -->
-								<include>org/apache/flink/stormcompatibility/api/*.class</include>
-								<include>org/apache/flink/stormcompatibility/wrappers/*.class</include>
-								<!-- Word Count -->
-								<include>org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.class</include>
-								<include>org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.class</include>
-								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
-							</includes>
-						</configuration>
-					</execution>
-
-					<!-- WordCount Storm topology-->
-					<!-- Example for whole topologies (ie, if FlinkTopologyBuilder is used) -->
-					<!-- We cannot use maven-jar-plugin because 'defaults.yaml' must be included in jar.
-					     However, we excluded 'defaults.yaml' in dependency-plugin to get clean Eclipse environment.
-					     Thus, 'defaults.yaml' is not available for maven-jar-plugin.
-					     Nevertheless, we register an empty jar with corresponding name, such that the final jar can be installed to local maven repository.
-					     We use maven-shade-plugin to build the actual jar (which will replace the empty jar). -->
-					<execution>
-						<id>WordCount-StormTopology</id>
-						<phase>package</phase>
-						<goals>
-							<goal>jar</goal>
-						</goals>
-						<configuration>
-							<finalName>WordCount</finalName>
-							<classifier>StormTopology</classifier>
-						</configuration>
-					</execution>
-
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- WordCount Storm topology-->
-			<!-- Cannot use maven-jar-plugin because 'defaults.yaml' must be included in jar -->
-			<!-- Build StormTopolgy jar to overwrite empty jar created with maven-jar-plugin. -->
-			<plugin>
-				<artifactId>maven-shade-plugin</artifactId>
-				<groupId>org.apache.maven.plugins</groupId>
-				<version>2.4.1</version>
-				<executions>
-					<execution>
-						<id>WordCount-StormTopology</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration>
-							<finalName>WordCount-StormTopology</finalName>
-
-							<artifactSet>
-								<includes>
-									<include>org.apache.storm:storm-core</include>
-									<!-- Storm's recursive dependencies -->
-									<include>org.yaml:snakeyaml</include>
-									<include>com.googlecode.json-simple:json-simple</include>
-									<include>org.apache.flink:flink-storm-compatibility-core</include>
-									<include>org.apache.flink:flink-storm-compatibility-examples</include>
-								</includes>
-							</artifactSet>
-							<filters>
-								<filter>
-									<artifact>org.apache.storm:storm-core</artifact>
-									<includes>
-										<include>defaults.yaml</include>
-										<include>backtype/storm/*.class</include>
-										<include>backtype/storm/topology/*.class</include>
-										<include>backtype/storm/spout/*.class</include>
-										<include>backtype/storm/task/*.class</include>
-										<include>backtype/storm/tuple/*.class</include>
-										<include>backtype/storm/generated/*.class</include>
-										<include>backtype/storm/metric/**/*.class</include>
-										<include>backtype/storm/utils/*.class</include>
-										<include>backtype/storm/serialization/*.class</include>
-										<include>org/apache/storm/curator/**/*.class</include>
-										<include>org/apache/thrift7/**/*.class</include>
-										<!-- Storm's recursive dependencies -->
-										<include>org/json/simple/**/*.class</include>
-										<include>org/yaml/snakeyaml/**/*.class</include>
-									</includes>
-								</filter>
-								<filter>
-									<artifact>org.apache.flink:flink-storm-compatibility-examples</artifact>
-									<includes>
-										<include>org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.class</include>
-										<include>org/apache/flink/stormcompatibility/wordcount/WordCountTopology.class</include>
-										<include>org/apache/flink/stormcompatibility/wordcount/stormoperators/*.class</include>
-										<include>org/apache/flink/stormcompatibility/util/*.class</include>
-										<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
-									</includes>
-								</filter>
-								<filter>
-									<artifact>org.apache.flink:flink-storm-compatibility-core</artifact>
-									<includes>
-										<include>org/apache/flink/stormcompatibility/api/*.class</include>
-										<include>org/apache/flink/stormcompatibility/wrappers/*.class</include>
-									</includes>
-								</filter>
-							</filters>
-							<transformers>
-								<transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
-									<mainClass>org.apache.flink.stormcompatibility.wordcount.StormWordCountRemoteBySubmitter</mainClass>
-								</transformer>
-							</transformers>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-
-		<pluginManagement>
-			<plugins>
-				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-				<plugin>
-					<groupId>org.eclipse.m2e</groupId>
-					<artifactId>lifecycle-mapping</artifactId>
-					<version>1.0.0</version>
-					<configuration>
-						<lifecycleMappingMetadata>
-							<pluginExecutions>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.maven.plugins</groupId>
-										<artifactId>maven-dependency-plugin</artifactId>
-										<versionRange>[2.9,)</versionRange>
-										<goals>
-											<goal>unpack</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-							</pluginExecutions>
-						</lifecycleMappingMetadata>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-
-	</build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java
deleted file mode 100644
index d8d620b..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationTopology.java
+++ /dev/null
@@ -1,128 +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 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;
-
-/**
- * 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}.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>StormExclamation[Local|RemoteByClient|RemoteBySubmitter] &lt;text path&gt;
- * &lt;result path&gt;</code><br/>
- * If no parameters are provided, the program is run with default data from
- * {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>construct a regular Storm topology as Flink program</li>
- * <li>make use of the FiniteStormSpout interface</li>
- * </ul>
- */
-public class ExclamationTopology {
-
-	public final static String spoutId = "source";
-	public final static String firstBoltId = "exclamation1";
-	public final static String secondBoltId = "exclamation2";
-	public final static String sinkId = "sink";
-	private final static OutputFormatter formatter = new SimpleOutputFormatter();
-
-	public static FlinkTopologyBuilder buildTopology() {
-		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
-
-		// get input data
-		if (fileInputOutput) {
-			// 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));
-		} else {
-			builder.setSpout(spoutId, new FiniteStormInMemorySpout(WordCountData.WORDS));
-		}
-
-		builder.setBolt(firstBoltId, new ExclamationBolt(), 3).shuffleGrouping(spoutId);
-		builder.setBolt(secondBoltId, new ExclamationBolt(), 2).shuffleGrouping(firstBoltId);
-
-		// 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(secondBoltId);
-		} else {
-			builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4)
-					.shuffleGrouping(secondBoltId);
-		}
-
-		return builder;
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileInputOutput = false;
-	private static String textPath;
-	private static String outputPath;
-	private static int exclamationNum = 3;
-
-	static int getExclamation() {
-		return exclamationNum;
-	}
-
-	static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileInputOutput = true;
-			if (args.length == 3) {
-				textPath = args[0];
-				outputPath = args[1];
-				exclamationNum = Integer.parseInt(args[2]);
-			} else {
-				System.err.println(
-						"Usage: StormExclamation[Local|RemoteByClient|RemoteBySubmitter] <text " +
-						"path> <result path>  <number of exclamation marks>");
-				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] <text path>" +
-					" <result path> <number of exclamation marks>");
-		}
-
-		return true;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java
deleted file mode 100644
index c8af3a6..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormBolt.java
+++ /dev/null
@@ -1,143 +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 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.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}.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>StormExclamationWithStormBolt &lt;text path&gt; &lt;result path&gt; &lt;number of exclamation marks&gt;</code><br/>
- * If no parameters are provided, the program is run with default data from
- * {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>use a Storm bolt within a Flink Streaming program</li>
- * </ul>
- */
-public class ExclamationWithStormBolt {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// set Storm configuration
-		StormConfig config = new StormConfig();
-		config.put(ExclamationBolt.EXCLAMATION_COUNT, new Integer(exclamationNum));
-		env.getConfig().setGlobalJobParameters(config);
-
-		// get input data
-		final DataStream<String> text = getTextDataStream(env);
-
-		final DataStream<String> exclaimed = text
-				.transform("StormBoltTokenizer",
-						TypeExtractor.getForObject(""),
-						new StormBoltWrapper<String, String>(new ExclamationBolt(),
-								new String[] { Utils.DEFAULT_STREAM_ID }))
-				.map(new ExclamationMap());
-
-		// emit result
-		if (fileOutput) {
-			exclaimed.writeAsText(outputPath);
-		} else {
-			exclaimed.print();
-		}
-
-		// execute program
-		env.execute("Streaming WordCount with Storm bolt tokenizer");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	private static class ExclamationMap implements MapFunction<String, String> {
-		private static final long serialVersionUID = 4614754344067170619L;
-
-		@Override
-		public String map(String value) throws Exception {
-			return value + "!!!";
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-	private static int exclamationNum = 3;
-
-	private static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 3) {
-				textPath = args[0];
-				outputPath = args[1];
-				exclamationNum = Integer.parseInt(args[2]);
-			} else {
-				System.err.println("Usage: ExclamationWithStormBolt <text path> <result path> <number of exclamation marks>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing ExclamationWithStormBolt example with built-in default data");
-			System.out.println("  Provide parameters to read input data from a file");
-			System.out.println("  Usage: ExclamationWithStormBolt <text path> <result path> <number of exclamation marks>");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			return env.readTextFile(textPath);
-		}
-
-		return env.fromElements(WordCountData.WORDS);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java
deleted file mode 100644
index 99c816d..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/ExclamationWithStormSpout.java
+++ /dev/null
@@ -1,150 +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 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.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}.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>StormExclamationWithStormSpout &lt;text path&gt; &lt;result path&gt;</code><br/>
- * If no parameters are provided, the program is run with default data from
- * {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>use a Storm spout within a Flink Streaming program</li>
- * <li>make use of the FiniteStormSpout interface</li>
- * </ul>
- */
-public class ExclamationWithStormSpout {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		final DataStream<String> text = getTextDataStream(env);
-
-		final DataStream<String> exclaimed = text
-				.map(new ExclamationMap())
-				.map(new ExclamationMap());
-
-		// emit result
-		if (fileOutput) {
-			exclaimed.writeAsText(outputPath);
-		} else {
-			exclaimed.print();
-		}
-
-		// execute program
-		env.execute("Streaming Exclamation with Storm spout source");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	private static class ExclamationMap implements MapFunction<String, String> {
-		private static final long serialVersionUID = -684993133807698042L;
-
-		@Override
-		public String map(String value) throws Exception {
-			return value + "!!!";
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: ExclamationWithStormSpout <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing ExclamationWithStormSpout example with built-in default " +
-					"data");
-			System.out.println("  Provide parameters to read input data from a file");
-			System.out.println("  Usage: ExclamationWithStormSpout <text path> <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			final String[] tokens = textPath.split(":");
-			final String inputFile = tokens[tokens.length - 1];
-
-			// set Storm configuration
-			StormConfig config = new StormConfig();
-			config.put(FiniteStormFileSpout.INPUT_FILE_PATH, inputFile);
-			env.getConfig().setGlobalJobParameters(config);
-
-			return env.addSource(
-					new FiniteStormSpoutWrapper<String>(new FiniteStormFileSpout(),
-							new String[] { Utils.DEFAULT_STREAM_ID }),
-							TypeExtractor.getForClass(String.class)).setParallelism(1);
-		}
-
-		return env.addSource(
-				new FiniteStormSpoutWrapper<String>(new FiniteStormInMemorySpout(
-						WordCountData.WORDS), new String[] { Utils.DEFAULT_STREAM_ID }),
-						TypeExtractor.getForClass(String.class)).setParallelism(1);
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java
deleted file mode 100644
index c87b3a5..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationLocal.java
+++ /dev/null
@@ -1,75 +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.utils.Utils;
-
-import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
-import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder;
-import org.apache.flink.stormcompatibility.excamation.stormoperators.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}.
- * <p/>
- * 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).
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>StormExclamationLocal &lt;text path&gt; &lt;result path&gt;</code><br/>
- * If no parameters are provided, the program is run with default data from
- * {@link org.apache.flink.examples.java.wordcount.util.WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>run a regular Storm program locally on Flink</li>
- * </ul>
- */
-public class StormExclamationLocal {
-
-	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 locally
-		Config conf = new Config();
-		conf.put(ExclamationBolt.EXCLAMATION_COUNT, ExclamationTopology.getExclamation());
-		final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster();
-		cluster.submitTopology(topologyId, conf, builder.createTopology());
-
-		Utils.sleep(10 * 1000);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteByClient.java
----------------------------------------------------------------------
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 0f64301..0000000
--- 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.
- * <p/>
- * 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).
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>StormExclamationRemoteByClient &lt;text path&gt; &lt;result path&gt;</code><br/>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>submit a regular Storm program to a local or remote Flink cluster.</li>
- * </ul>
- */
-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);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/StormExclamationRemoteBySubmitter.java
----------------------------------------------------------------------
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 d580520..0000000
--- 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.
- * <p/>
- * This example shows how to submit the program via Java as well as Flink's command line client (ie, bin/flink).
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>StormExclamationRemoteByClient &lt;text path&gt; &lt;result path&gt;</code><br/>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>submit a regular Storm program to a local or remote Flink cluster.</li>
- * </ul>
- */
-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);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/stormoperators/ExclamationBolt.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/stormoperators/ExclamationBolt.java
deleted file mode 100644
index 2709eff..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/excamation/stormoperators/ExclamationBolt.java
+++ /dev/null
@@ -1,75 +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.stormoperators;
-
-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.tuple.Values;
-
-import java.util.Map;
-
-public class ExclamationBolt implements IRichBolt {
-	private final static long serialVersionUID = -6364882114201311380L;
-
-	public final static String EXCLAMATION_COUNT = "exclamation.count";
-
-	private OutputCollector collector;
-	private String exclamation;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
-		this.collector = collector;
-
-		Object count = conf.get(EXCLAMATION_COUNT);
-		if (count != null) {
-			int exclamationNum = (Integer) count;
-			StringBuilder builder = new StringBuilder();
-			for (int index = 0; index < exclamationNum; ++index) {
-				builder.append('!');
-			}
-			this.exclamation = builder.toString();
-		} else {
-			this.exclamation = "!";
-		}
-	}
-
-	@Override
-	public void cleanup() {
-	}
-
-	@Override
-	public void execute(Tuple tuple) {
-		collector.emit(tuple, new Values(tuple.getString(0) + this.exclamation));
-	}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields("word"));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/SingleJoinTopology.java
----------------------------------------------------------------------
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 79c7125..0000000
--- 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* <result path>");
-				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* <result path>");
-		}
-		return true;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/StormSingleJoinLocal.java
----------------------------------------------------------------------
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 d70914a..0000000
--- 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();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/AgeSpout.java
----------------------------------------------------------------------
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 49761c3..0000000
--- 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);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/GenderSpout.java
----------------------------------------------------------------------
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 238b6db..0000000
--- 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);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/singlejoin/stormoperators/SingleJoinBolt.java
----------------------------------------------------------------------
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 cd53140..0000000
--- 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<List<Object>, Map<GlobalStreamId, Tuple>> pending;
-	Map<String, GlobalStreamId> fieldLocations;
-
-	public SingleJoinBolt(Fields outFields) {
-		this.outFields = outFields;
-	}
-
-	@SuppressWarnings({"rawtypes", "null"})
-	@Override
-	public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
-		fieldLocations = new HashMap<String, GlobalStreamId>();
-		this.collector = collector;
-		int timeout = 100;
-		pending = new TimeCacheMap<List<Object>, Map<GlobalStreamId, Tuple>>(timeout, new ExpireCallback());
-		// numSources = context.getThisSources().size();
-		Set<String> idFields = null;
-		for (GlobalStreamId source : context.getThisSources().keySet()) {
-			Fields fields = context.getComponentOutputFields(source.get_componentId(), source.get_streamId());
-			Set<String> setFields = new HashSet<String>(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<String>(idFields));
-
-		if (fieldLocations.size() != outFields.size()) {
-			throw new RuntimeException("Cannot find all outfields among sources");
-		}
-	}
-
-	@Override
-	public void execute(Tuple tuple) {
-		List<Object> id = tuple.select(idFields);
-		GlobalStreamId streamId = new GlobalStreamId(tuple.getSourceComponent(), tuple.getSourceStreamId());
-		if (!pending.containsKey(id)) {
-			pending.put(id, new HashMap<GlobalStreamId, Tuple>());
-		}
-		Map<GlobalStreamId, Tuple> 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<Object> joinResult = new ArrayList<Object>();
-			for (String outField : outFields) {
-				GlobalStreamId loc = fieldLocations.get(outField);
-				joinResult.add(parts.get(loc).getValueByField(outField));
-			}
-			collector.emit(new ArrayList<Tuple>(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<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-	private class ExpireCallback implements TimeCacheMap.ExpiredCallback<List<Object>, Map<GlobalStreamId, Tuple>> {
-		@Override
-		public void expire(List<Object> id, Map<GlobalStreamId, Tuple> tuples) {
-			for (Tuple tuple : tuples.values()) {
-				collector.fail(tuple);
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java
deleted file mode 100644
index 18251d4..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/SpoutSplitExample.java
+++ /dev/null
@@ -1,102 +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.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.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.
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>handle multiple output stream of a spout</li>
- * <li>accessing each stream by .split(...) and .select(...)</li>
- * <li>strip wrapper data type SplitStreamType for furhter processing in Flink</li>
- * </ul>
- * <p/>
- * This example would work the same way for multiple bolt output streams.
- */
-public class SpoutSplitExample {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		String[] rawOutputs = new String[] { RandomSpout.EVEN_STREAM, RandomSpout.ODD_STREAM };
-
-		final DataStream<SplitStreamType<Integer>> numbers = env.addSource(
-				new StormSpoutWrapper<SplitStreamType<Integer>>(new RandomSpout(true, 0),
-						rawOutputs), TypeExtractor.getForObject(new SplitStreamType<Integer>()));
-
-		SplitStream<SplitStreamType<Integer>> splitStream = numbers
-				.split(new FlinkStormStreamSelector<Integer>());
-
-		DataStream<SplitStreamType<Integer>> evenStream = splitStream.select(RandomSpout.EVEN_STREAM);
-		DataStream<SplitStreamType<Integer>> oddStream = splitStream.select(RandomSpout.ODD_STREAM);
-
-		evenStream.map(new SplitStreamMapper<Integer>()).returns(Integer.class).map(new Enrich("even")).print();
-		oddStream.transform("oddBolt",
-				TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
-				new StormBoltWrapper<SplitStreamType<Integer>, Tuple2<String, Integer>>(
-						new VerifyAndEnrichBolt(false)))
-						.print();
-
-		// execute program
-		env.execute("Spout split stream example");
-	}
-
-	// *************************************************************************
-	//     USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Same as {@link VerifyAndEnrichBolt}.
-	 */
-	private final static class Enrich implements MapFunction<Integer, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 5213888269197438892L;
-		private final Tuple2<String, Integer> out;
-
-		public Enrich(String token) {
-			this.out = new Tuple2<String, Integer>(token, 0);
-		}
-
-		@Override
-		public Tuple2<String, Integer> map(Integer value) throws Exception {
-			this.out.setField(value, 1);
-			return this.out;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.java
deleted file mode 100644
index 75d710e..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/RandomSpout.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.split.stormoperators;
-
-import java.util.Map;
-import java.util.Random;
-
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichSpout;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
-
-public class RandomSpout extends BaseRichSpout {
-	private static final long serialVersionUID = -3978554318742509334L;
-
-	public static final String EVEN_STREAM = "even";
-	public static final String ODD_STREAM = "odd";
-
-	private final boolean split;
-	private Random r = new Random();
-	private SpoutOutputCollector collector;
-
-	public RandomSpout(boolean split, long seed) {
-		this.split = split;
-		this.r = new Random(seed);
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void nextTuple() {
-		int i = r.nextInt();
-		if (split) {
-			if (i % 2 == 0) {
-				this.collector.emit(EVEN_STREAM, new Values(i));
-			} else {
-				this.collector.emit(ODD_STREAM, new Values(i));
-			}
-		} else {
-			this.collector.emit(new Values(i));
-		}
-	}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {
-		Fields schema = new Fields("number");
-		if (split) {
-			declarer.declareStream(EVEN_STREAM, schema);
-			declarer.declareStream(ODD_STREAM, schema);
-		} else {
-			declarer.declare(schema);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java
deleted file mode 100644
index 5853705..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/split/stormoperators/VerifyAndEnrichBolt.java
+++ /dev/null
@@ -1,61 +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.split.stormoperators;
-
-import java.util.Map;
-
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-
-public class VerifyAndEnrichBolt extends BaseRichBolt {
-	private static final long serialVersionUID = -7277395570966328721L;
-
-	private final boolean evenOrOdd; // true: even -- false: odd
-	private final String token;
-	private OutputCollector collector;
-
-	public VerifyAndEnrichBolt(boolean evenOrOdd) {
-		this.evenOrOdd = evenOrOdd;
-		this.token = evenOrOdd ? "even" : "odd";
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void execute(Tuple input) {
-		if ((input.getInteger(0) % 2 == 0) != this.evenOrOdd) {
-			throw new RuntimeException("Invalid number detected.");
-		}
-		this.collector.emit(new Values(this.token, input.getInteger(0)));
-	}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields("evenOrOdd", "number"));
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormBoltSink.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormBoltSink.java
deleted file mode 100644
index b121744..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormBoltSink.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.util;
-
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IRichBolt;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Tuple;
-
-import java.util.Map;
-
-/**
- * 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 {
-	private static final long serialVersionUID = -1626323806848080430L;
-
-	private StringBuilder lineBuilder;
-	private String prefix = "";
-	private final OutputFormatter formatter;
-
-	public AbstractStormBoltSink(final OutputFormatter formatter) {
-		this.formatter = formatter;
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public final void prepare(final Map stormConf, final TopologyContext context,
-			final OutputCollector collector) {
-		this.prepareSimple(stormConf, context);
-		if (context.getComponentCommon(context.getThisComponentId()).get_parallelism_hint() > 1) {
-			this.prefix = context.getThisTaskId() + "> ";
-		}
-	}
-
-	protected abstract void prepareSimple(final Map<?, ?> stormConf, final TopologyContext context);
-
-	@Override
-	public final void execute(final Tuple input) {
-		this.lineBuilder = new StringBuilder();
-		this.lineBuilder.append(this.prefix);
-		this.lineBuilder.append(this.formatter.format(input));
-		this.writeExternal(this.lineBuilder.toString());
-	}
-
-	protected abstract void writeExternal(final String line);
-
-	@Override
-	public void cleanup() {/* nothing to do */}
-
-	@Override
-	public final void declareOutputFields(final OutputFieldsDeclarer declarer) {/* nothing to do */}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormSpout.java
deleted file mode 100644
index 4739a2c..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/AbstractStormSpout.java
+++ /dev/null
@@ -1,70 +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.util;
-
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IRichSpout;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-
-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}.
- */
-public abstract class AbstractStormSpout implements IRichSpout {
-	private static final long serialVersionUID = 8876828403487806771L;
-
-	public final static String ATTRIBUTE_LINE = "line";
-
-	protected SpoutOutputCollector collector;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void close() {/* noting to do */}
-
-	@Override
-	public void activate() {/* noting to do */}
-
-	@Override
-	public void deactivate() {/* noting to do */}
-
-	@Override
-	public void ack(final Object msgId) {/* noting to do */}
-
-	@Override
-	public void fail(final Object msgId) {/* noting to do */}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields(ATTRIBUTE_LINE));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java
----------------------------------------------------------------------
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
deleted file mode 100644
index d3776fb..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java
+++ /dev/null
@@ -1,77 +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.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;
-
-/**
- * Implements a Storm 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 {
-	private static final long serialVersionUID = -1472978008607215864L;
-
-	private String line;
-	private boolean newLineRead;
-
-	public FiniteStormFileSpout() {}
-
-	public FiniteStormFileSpout(String path) {
-		super(path);
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
-		super.open(conf, context, collector);
-		newLineRead = false;
-	}
-
-	@Override
-	public void nextTuple() {
-		this.collector.emit(new Values(line));
-		newLineRead = false;
-	}
-
-	/**
-	 * Can be called before nextTuple() any times including 0.
-	 */
-	@Override
-	public boolean reachedEnd() {
-		try {
-			readLine();
-		} catch (IOException e) {
-			throw new RuntimeException("Exception occured while reading file " + path);
-		}
-		return line == null;
-	}
-
-	private void readLine() throws IOException {
-		if (!newLineRead) {
-			line = reader.readLine();
-			newLineRead = true;
-		}
-	}
-
-}


[08/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 5efff66..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java
+++ /dev/null
@@ -1,40 +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.util;
-
-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.
- */
-public class FiniteStormInMemorySpout extends StormInMemorySpout<String> implements
-		FiniteStormSpout {
-	private static final long serialVersionUID = -4008858647468647019L;
-
-	public FiniteStormInMemorySpout(String[] source) {
-		super(source);
-	}
-
-	@Override
-	public boolean reachedEnd() {
-		return counter >= source.length;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java
deleted file mode 100644
index ec9adfe..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/OutputFormatter.java
+++ /dev/null
@@ -1,36 +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.util;
-
-import backtype.storm.tuple.Tuple;
-
-import java.io.Serializable;
-
-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
-	 * @return The string result of the formatting
-	 */
-	public String format(Tuple input);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java
deleted file mode 100644
index 0702e94..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/SimpleOutputFormatter.java
+++ /dev/null
@@ -1,42 +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.util;
-
-import backtype.storm.tuple.Tuple;
-
-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.
-	 *
-	 * @param input
-	 * 		The tuple to be formatted
-	 * @return The string result of the formatting
-	 */
-	@Override
-	public String format(final Tuple input) {
-		if (input.getValues().size() != 1) {
-			throw new RuntimeException("The output is not raw");
-		}
-		return input.getValue(0).toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltFileSink.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltFileSink.java
deleted file mode 100644
index ee8dca4..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltFileSink.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.util;
-
-import backtype.storm.task.TopologyContext;
-
-import java.io.BufferedWriter;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * 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 {
-	private static final long serialVersionUID = 2014027288631273666L;
-
-	private final String path;
-	private BufferedWriter writer;
-
-	public StormBoltFileSink(final String path) {
-		this(path, new SimpleOutputFormatter());
-	}
-
-	public StormBoltFileSink(final String path, final OutputFormatter formatter) {
-		super(formatter);
-		this.path = path;
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepareSimple(final Map stormConf, final TopologyContext context) {
-		try {
-			this.writer = new BufferedWriter(new FileWriter(this.path));
-		} catch (final IOException e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	@Override
-	public void writeExternal(final String line) {
-		try {
-			this.writer.write(line + "\n");
-		} catch (final IOException e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	@Override
-	public void cleanup() {
-		if (this.writer != null) {
-			try {
-				this.writer.close();
-			} catch (final IOException e) {
-				throw new RuntimeException(e);
-			}
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltPrintSink.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltPrintSink.java
deleted file mode 100644
index 3bf49d0..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormBoltPrintSink.java
+++ /dev/null
@@ -1,45 +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.util;
-
-import backtype.storm.task.TopologyContext;
-
-import java.util.Map;
-
-/**
- * Implements a sink that prints the received data to {@code stdout}.
- */
-public final class StormBoltPrintSink extends AbstractStormBoltSink {
-	private static final long serialVersionUID = -6650011223001009519L;
-
-	public StormBoltPrintSink(OutputFormatter formatter) {
-		super(formatter);
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepareSimple(final Map stormConf, final TopologyContext context) {
-		/* nothing to do */
-	}
-
-	@Override
-	public void writeExternal(final String line) {
-		System.out.println(line);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormFileSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormFileSpout.java
deleted file mode 100644
index 0611e37..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormFileSpout.java
+++ /dev/null
@@ -1,88 +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.util;
-
-import backtype.storm.spout.SpoutOutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.tuple.Values;
-
-import java.io.BufferedReader;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.Map;
-
-/**
- * Implements a Storm Spout that reads data from a given local file.
- */
-public class StormFileSpout extends AbstractStormSpout {
-	private static final long serialVersionUID = -6996907090003590436L;
-
-	public final static String INPUT_FILE_PATH = "input.path";
-
-	protected String path = null;
-	protected BufferedReader reader;
-
-	public StormFileSpout() {}
-
-	public StormFileSpout(final String path) {
-		this.path = path;
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
-		super.open(conf, context, collector);
-
-		Object configuredPath = conf.get(INPUT_FILE_PATH);
-		if(configuredPath != null) {
-			this.path = (String)configuredPath;
-		}
-
-		try {
-			this.reader = new BufferedReader(new FileReader(this.path));
-		} catch (final FileNotFoundException e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	@Override
-	public void close() {
-		if (this.reader != null) {
-			try {
-				this.reader.close();
-			} catch (final IOException e) {
-				throw new RuntimeException(e);
-			}
-		}
-	}
-
-	@Override
-	public void nextTuple() {
-		String line;
-		try {
-			line = this.reader.readLine();
-			if (line != null) {
-				this.collector.emit(new Values(line));
-			}
-		} catch (final IOException e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormInMemorySpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormInMemorySpout.java
deleted file mode 100644
index f6ae622..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/StormInMemorySpout.java
+++ /dev/null
@@ -1,42 +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.util;
-
-import backtype.storm.tuple.Values;
-
-/**
- * Implements a Storm Spout that reads data from an in.
- */
-public class StormInMemorySpout<T> extends AbstractStormSpout {
-	private static final long serialVersionUID = -4008858647468647019L;
-
-	protected T[] source;
-	protected int counter = 0;
-
-	public StormInMemorySpout(T[] source) {
-		this.source = source;
-	}
-
-	@Override
-	public void nextTuple() {
-		if (this.counter < source.length) {
-			this.collector.emit(new Values(source[this.counter++]));
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/TupleOutputFormatter.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/TupleOutputFormatter.java
deleted file mode 100644
index 6419ee3..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/TupleOutputFormatter.java
+++ /dev/null
@@ -1,38 +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.util;
-
-import backtype.storm.tuple.Tuple;
-
-public class TupleOutputFormatter implements OutputFormatter {
-	private static final long serialVersionUID = -599665757723851761L;
-
-	@Override
-	public String format(final Tuple input) {
-		final StringBuilder stringBuilder = new StringBuilder();
-		stringBuilder.append("(");
-		for (final Object attribute : input.getValues()) {
-			stringBuilder.append(attribute);
-			stringBuilder.append(",");
-		}
-		stringBuilder.replace(stringBuilder.length() - 1, stringBuilder.length(), ")");
-		return stringBuilder.toString();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java
deleted file mode 100644
index 6f7b6fb..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCount.java
+++ /dev/null
@@ -1,122 +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.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.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}.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>use a Storm bolt within a Flink Streaming program.</li>
- * </ul>
- */
-public class BoltTokenizerWordCount {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		final DataStream<String> text = getTextDataStream(env);
-
-		final DataStream<Tuple2<String, Integer>> 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",
-						TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
-						new StormBoltWrapper<String, Tuple2<String, Integer>>(new StormBoltTokenizer()))
-				// split up the lines in pairs (2-tuples) containing: (word,1)
-				// group by the tuple field "0" and sum up tuple field "1"
-				.keyBy(0).sum(1);
-
-		// emit result
-		if (fileOutput) {
-			counts.writeAsText(outputPath);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("Streaming WordCount with Storm bolt tokenizer");
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: BoltTokenizerWordCount <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing BoltTokenizerWordCount example with built-in default data");
-			System.out.println("  Provide parameters to read input data from a file");
-			System.out.println("  Usage: BoltTokenizerWordCount <text path> <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			return env.readTextFile(textPath);
-		}
-
-		return env.fromElements(WordCountData.WORDS);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java
deleted file mode 100644
index 300f5bc..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojo.java
+++ /dev/null
@@ -1,135 +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.wordcount;
-
-import backtype.storm.topology.IRichBolt;
-
-import org.apache.flink.api.java.io.CsvInputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-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.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.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>how to access attributes by name for POJO type input streams
- * </ul>
- */
-public class BoltTokenizerWordCountPojo {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		final DataStream<Sentence> text = getTextDataStream(env);
-
-		final DataStream<Tuple2<String, Integer>> 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",
-						TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
-						new StormBoltWrapper<Sentence, Tuple2<String, Integer>>(
-								new StormBoltTokenizerByName()))
-				// split up the lines in pairs (2-tuples) containing: (word,1)
-				// group by the tuple field "0" and sum up tuple field "1"
-				.keyBy(0).sum(1);
-
-		// emit result
-		if (fileOutput) {
-			counts.writeAsText(outputPath);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("Streaming WordCount with Storm bolt tokenizer");
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: BoltTokenizerWordCount <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing BoltTokenizerWordCount example with built-in default data");
-			System.out.println("  Provide parameters to read input data from a file");
-			System.out.println("  Usage: BoltTokenizerWordCount <text path> <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<Sentence> getTextDataStream(final StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			PojoTypeInfo<Sentence> sourceType = (PojoTypeInfo)TypeExtractor
-					.getForObject(new Sentence(""));
-			return env.createInput(new CsvInputFormat<Sentence>(new Path(
-					textPath), CsvInputFormat.DEFAULT_LINE_DELIMITER,
-					CsvInputFormat.DEFAULT_LINE_DELIMITER, sourceType),
-					sourceType);
-		}
-
-		return env.fromElements(WordCountDataPojos.SENTENCES);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java
deleted file mode 100644
index ed01181..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNames.java
+++ /dev/null
@@ -1,138 +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.wordcount;
-
-import backtype.storm.topology.IRichBolt;
-import backtype.storm.tuple.Fields;
-
-import org.apache.flink.api.java.io.CsvInputFormat;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-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.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.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>how to access attributes by name for {@link Tuple} type input streams
- * </ul>
- */
-public class BoltTokenizerWordCountWithNames {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		final DataStream<Tuple1<String>> text = getTextDataStream(env);
-
-		final DataStream<Tuple2<String, Integer>> 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",
-						TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
-						new StormBoltWrapper<Tuple1<String>, Tuple2<String, Integer>>(
-								new StormBoltTokenizerByName(), new Fields("sentence")))
-				// split up the lines in pairs (2-tuples) containing: (word,1)
-				// group by the tuple field "0" and sum up tuple field "1"
-				.keyBy(0).sum(1);
-
-		// emit result
-		if (fileOutput) {
-			counts.writeAsText(outputPath);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("Streaming WordCount with Storm bolt tokenizer");
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: BoltTokenizerWordCount <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing BoltTokenizerWordCount example with built-in default data");
-			System.out.println("  Provide parameters to read input data from a file");
-			System.out.println("  Usage: BoltTokenizerWordCount <text path> <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<Tuple1<String>> getTextDataStream(final StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			TupleTypeInfo<Tuple1<String>> sourceType = (TupleTypeInfo<Tuple1<String>>)TypeExtractor
-					.getForObject(new Tuple1<String>(""));
-			return env.createInput(new CsvInputFormat<Tuple1<String>>(new Path(
-					textPath), CsvInputFormat.DEFAULT_LINE_DELIMITER,
-					CsvInputFormat.DEFAULT_LINE_DELIMITER, sourceType),
-					sourceType);
-		}
-
-		return env.fromElements(WordCountDataTuple.TUPLES);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java
deleted file mode 100644
index 21d7811..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCount.java
+++ /dev/null
@@ -1,157 +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.wordcount;
-
-import backtype.storm.topology.IRichSpout;
-import backtype.storm.utils.Utils;
-
-import org.apache.flink.api.common.functions.FlatMapFunction;
-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.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}.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>use a Storm spout within a Flink Streaming program.</li>
- * </ul>
- */
-public class SpoutSourceWordCount {
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!parseParameters(args)) {
-			return;
-		}
-
-		// set up the execution environment
-		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-		// get input data
-		final DataStream<String> text = getTextDataStream(env);
-
-		final DataStream<Tuple2<String, Integer>> counts =
-				// split up the lines in pairs (2-tuples) containing: (word,1)
-				text.flatMap(new Tokenizer())
-				// group by the tuple field "0" and sum up tuple field "1"
-				.keyBy(0).sum(1);
-
-		// emit result
-		if (fileOutput) {
-			counts.writeAsText(outputPath);
-		} else {
-			counts.print();
-		}
-
-		// execute program
-		env.execute("Streaming WordCount with Storm spout source");
-	}
-
-	// *************************************************************************
-	// USER FUNCTIONS
-	// *************************************************************************
-
-	/**
-	 * Implements the string tokenizer that splits sentences into words as a user-defined FlatMapFunction. The function
-	 * takes a line (String) and splits it into multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
-	 */
-	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void flatMap(final String value, final Collector<Tuple2<String, Integer>> out) throws Exception {
-			// normalize and split the line
-			final String[] tokens = value.toLowerCase().split("\\W+");
-
-			// emit the pairs
-			for (final String token : tokens) {
-				if (token.length() > 0) {
-					out.collect(new Tuple2<String, Integer>(token, 1));
-				}
-			}
-		}
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	private static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: SpoutSourceWordCount <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing SpoutSourceWordCount example with built-in default data");
-			System.out.println("  Provide parameters to read input data from a file");
-			System.out.println("  Usage: SpoutSourceWordCount <text path> <result path>");
-		}
-		return true;
-	}
-
-	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
-		if (fileOutput) {
-			// read the text file from given input path
-			final String[] tokens = textPath.split(":");
-			final String localFile = tokens[tokens.length - 1];
-			return env.addSource(
-					new StormFiniteSpoutWrapper<String>(new StormWordCountFileSpout(localFile),
-							new String[] { Utils.DEFAULT_STREAM_ID }),
-					TypeExtractor.getForClass(String.class)).setParallelism(1);
-		}
-
-		return env.addSource(
-				new StormFiniteSpoutWrapper<String>(new StormWordCountInMemorySpout(),
-						new String[] { Utils.DEFAULT_STREAM_ID }),
-				TypeExtractor.getForClass(String.class)).setParallelism(1);
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java
deleted file mode 100644
index 836c8e9..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocal.java
+++ /dev/null
@@ -1,75 +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.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;
-
-/**
- * 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}.
- * <p/>
- * This example shows how to run program directly within Java, thus it cannot be used to submit a {@link StormTopology}
- * via Flink command line clients (ie, bin/flink).
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>run a regular Storm program locally on Flink</li>
- * </ul>
- */
-public class StormWordCountLocal {
-	public final static String topologyId = "Streaming WordCount";
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!WordCountTopology.parseParameters(args)) {
-			return;
-		}
-
-		// build Topology the Storm way
-		final FlinkTopologyBuilder builder = WordCountTopology.buildTopology();
-
-		// execute program locally
-		final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster();
-		cluster.submitTopology(topologyId, null, builder.createTopology());
-
-		Utils.sleep(10 * 1000);
-
-		// TODO kill does no do anything so far
-		cluster.killTopology(topologyId);
-		cluster.shutdown();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountNamedLocal.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountNamedLocal.java
deleted file mode 100644
index f51afab..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountNamedLocal.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.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;
-
-/**
- * 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
- * input tuples by name instead of index.
- * <p/>
- * This example shows how to run program directly within Java, thus it cannot be used to submit a {@link StormTopology}
- * via Flink command line clients (ie, bin/flink).
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>run a regular Storm program locally on Flink
- * </ul>
- */
-public class StormWordCountNamedLocal {
-	public final static String topologyId = "Streaming WordCountName";
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!WordCountTopology.parseParameters(args)) {
-			return;
-		}
-
-		// build Topology the Storm way
-		final FlinkTopologyBuilder builder = WordCountTopology.buildTopology(false);
-
-		// execute program locally
-		final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster();
-		cluster.submitTopology(topologyId, null, builder.createTopology());
-
-		Utils.sleep(10 * 1000);
-
-		// TODO kill does no do anything so far
-		cluster.killTopology(topologyId);
-		cluster.shutdown();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java
deleted file mode 100644
index 3c79eda..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteByClient.java
+++ /dev/null
@@ -1,85 +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.wordcount;
-
-import backtype.storm.Config;
-import backtype.storm.generated.AlreadyAliveException;
-import backtype.storm.generated.InvalidTopologyException;
-import backtype.storm.generated.NotAliveException;
-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;
-
-/**
- * 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 cluster similar to {@link NimbusClient}. The Flink cluster can be local or remote.
- * <p/>
- * 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).
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>submit a regular Storm program to a local or remote Flink cluster.</li>
- * </ul>
- */
-public class StormWordCountRemoteByClient {
-	public final static String topologyId = "Streaming WordCount";
-	private final static String uploadedJarLocation = "target/WordCount-StormTopology.jar";
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws AlreadyAliveException, InvalidTopologyException,
-			NotAliveException {
-
-		if (!WordCountTopology.parseParameters(args)) {
-			return;
-		}
-
-		// build Topology the Storm way
-		final FlinkTopologyBuilder builder = WordCountTopology.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);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.java
deleted file mode 100644
index de84f55..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountRemoteBySubmitter.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.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;
-
-/**
- * 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 cluster similar to {@link StormSubmitter}. The Flink cluster can be local or remote.
- * <p/>
- * This example shows how to submit the program via Java as well as Flink's command line client (ie, bin/flink).
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>submit a regular Storm program to a local or remote Flink cluster.</li>
- * </ul>
- */
-public class StormWordCountRemoteBySubmitter {
-	public final static String topologyId = "Streaming WordCount";
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!WordCountTopology.parseParameters(args)) {
-			return;
-		}
-
-		// build Topology the Storm way
-		final FlinkTopologyBuilder builder = WordCountTopology.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/WordCount-StormTopology.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);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java
deleted file mode 100644
index 45be821..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/WordCountTopology.java
+++ /dev/null
@@ -1,135 +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.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;
-
-/**
- * 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}.
- * <p/>
- * <p/>
- * The input is a plain text file with lines separated by newline characters.
- * <p/>
- * <p/>
- * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
- * If no parameters are provided, the program is run with default data from {@link WordCountData}.
- * <p/>
- * <p/>
- * This example shows how to:
- * <ul>
- * <li>how to construct a regular Storm topology as Flink program</li>
- * </ul>
- */
-public class WordCountTopology {
-	public final static String spoutId = "source";
-	public final static String tokenierzerId = "tokenizer";
-	public final static String counterId = "counter";
-	public final static String sinkId = "sink";
-	private final static OutputFormatter formatter = new TupleOutputFormatter();
-
-	public static FlinkTopologyBuilder buildTopology() {
-		return buildTopology(true);
-	}
-
-	public static FlinkTopologyBuilder buildTopology(boolean indexOrName) {
-
-		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
-
-		// get input data
-		if (fileInputOutput) {
-			// 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));
-		} else {
-			builder.setSpout(spoutId, new StormWordCountInMemorySpout());
-		}
-
-		if (indexOrName) {
-			// split up the lines in pairs (2-tuples) containing: (word,1)
-			builder.setBolt(tokenierzerId, new StormBoltTokenizer(), 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));
-		} else {
-			// split up the lines in pairs (2-tuples) containing: (word,1)
-			builder.setBolt(tokenierzerId, new StormBoltTokenizerByName(), 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));
-		}
-
-		// 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(counterId);
-		} else {
-			builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4).shuffleGrouping(counterId);
-		}
-
-		return builder;
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static boolean fileInputOutput = false;
-	private static String textPath;
-	private static String outputPath;
-
-	static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			fileInputOutput = true;
-			if (args.length == 2) {
-				textPath = args[0];
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: StormWordCount* <text path> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing StormWordCount* example with built-in default data");
-			System.out.println("  Provide parameters to read input data from a file");
-			System.out.println("  Usage: StormWordCount* <text path> <result path>");
-		}
-
-		return true;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounter.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounter.java
deleted file mode 100644
index 1544c19..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounter.java
+++ /dev/null
@@ -1,88 +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.wordcount.stormoperators;
-
-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.tuple.Values;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Implements the word counter that the occurrence of each unique word. The bolt takes a pair (input tuple schema:
- * {@code <String,Integer>}) and sums the given word count for each unique word (output tuple schema:
- * {@code <String,Integer>} ).
- */
-public class StormBoltCounter implements IRichBolt {
-	private static final long serialVersionUID = 399619605462625934L;
-
-	public static final String ATTRIBUTE_WORD = "word";
-	public static final String ATTRIBUTE_COUNT = "count";
-
-	private final HashMap<String, Count> counts = new HashMap<String, Count>();
-	private OutputCollector collector;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void execute(final Tuple input) {
-		final String word = input.getString(StormBoltTokenizer.ATTRIBUTE_WORD_INDEX);
-
-		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);
-
-		this.collector.emit(new Values(word, currentCount.count));
-	}
-
-	@Override
-	public void cleanup() {/* nothing to do */}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-	/**
-	 * A counter helper to emit immutable tuples to the given stormCollector and avoid unnecessary object
-	 * creating/deletion.
-	 */
-	private static final class Count {
-		public int count;
-
-		public Count() {/* nothing to do */}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounterByName.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounterByName.java
deleted file mode 100644
index bf940c3..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltCounterByName.java
+++ /dev/null
@@ -1,88 +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.wordcount.stormoperators;
-
-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.tuple.Values;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Implements the word counter that the occurrence of each unique word. The bolt takes a pair (input tuple schema:
- * {@code <String,Integer>}) and sums the given word count for each unique word (output tuple schema:
- * {@code <String,Integer>} ).
- */
-public class StormBoltCounterByName implements IRichBolt {
-	private static final long serialVersionUID = 399619605462625934L;
-
-	public static final String ATTRIBUTE_WORD = "word";
-	public static final String ATTRIBUTE_COUNT = "count";
-
-	private final HashMap<String, Count> counts = new HashMap<String, Count>();
-	private OutputCollector collector;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void execute(final Tuple input) {
-		final String word = input.getStringByField(StormBoltTokenizer.ATTRIBUTE_WORD);
-
-		Count currentCount = this.counts.get(word);
-		if (currentCount == null) {
-			currentCount = new Count();
-			this.counts.put(word, currentCount);
-		}
-		currentCount.count += input.getIntegerByField(StormBoltTokenizer.ATTRIBUTE_COUNT);
-
-		this.collector.emit(new Values(word, currentCount.count));
-	}
-
-	@Override
-	public void cleanup() {/* nothing to do */}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-	/**
-	 * A counter helper to emit immutable tuples to the given stormCollector and avoid unnecessary object
-	 * creating/deletion.
-	 */
-	private static final class Count {
-		public int count;
-
-		public Count() {/* nothing to do */}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.java
deleted file mode 100644
index dfb3e37..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizer.java
+++ /dev/null
@@ -1,78 +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.wordcount.stormoperators;
-
-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.tuple.Values;
-
-import java.util.Map;
-
-/**
- * Implements the string tokenizer that splits sentences into words as a Storm bolt. The bolt takes a line (input tuple
- * schema: {@code <String>}) and splits it into multiple pairs in the form of "(word,1)" (output tuple schema:
- * {@code <String,Integer>}).
- * <p>
- * Same as {@link StormBoltTokenizerByName}, but accesses input attribute by index (instead of name).
- */
-public final class StormBoltTokenizer implements IRichBolt {
-	private static final long serialVersionUID = -8589620297208175149L;
-
-	public static final String ATTRIBUTE_WORD = "word";
-	public static final String ATTRIBUTE_COUNT = "count";
-
-	public static final int ATTRIBUTE_WORD_INDEX = 0;
-	public static final int ATTRIBUTE_COUNT_INDEX = 1;
-
-	private OutputCollector collector;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void execute(final Tuple input) {
-		final String[] tokens = input.getString(0).toLowerCase().split("\\W+");
-
-		for (final String token : tokens) {
-			if (token.length() > 0) {
-				this.collector.emit(new Values(token, 1));
-			}
-		}
-	}
-
-	@Override
-	public void cleanup() {/* nothing to do */}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizerByName.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizerByName.java
deleted file mode 100644
index 8796b95..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormBoltTokenizerByName.java
+++ /dev/null
@@ -1,78 +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.wordcount.stormoperators;
-
-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.tuple.Values;
-
-import java.util.Map;
-
-/**
- * Implements the string tokenizer that splits sentences into words as a Storm bolt. The bolt takes a line (input tuple
- * schema: {@code <String>}) and splits it into multiple pairs in the form of "(word,1)" (output tuple schema:
- * {@code <String,Integer>}).
- * <p>
- * Same as {@link StormBoltTokenizer}, but accesses input attribute by name (instead of index).
- */
-public final class StormBoltTokenizerByName implements IRichBolt {
-	private static final long serialVersionUID = -8589620297208175149L;
-
-	public static final String ATTRIBUTE_WORD = "word";
-	public static final String ATTRIBUTE_COUNT = "count";
-
-	public static final int ATTRIBUTE_WORD_INDEX = 0;
-	public static final int ATTRIBUTE_COUNT_INDEX = 1;
-
-	private OutputCollector collector;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(final Map stormConf, final TopologyContext context, final OutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void execute(final Tuple input) {
-		final String[] tokens = input.getStringByField("sentence").toLowerCase().split("\\W+");
-
-		for (final String token : tokens) {
-			if (token.length() > 0) {
-				this.collector.emit(new Values(token, 1));
-			}
-		}
-	}
-
-	@Override
-	public void cleanup() {/* nothing to do */}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields(ATTRIBUTE_WORD, ATTRIBUTE_COUNT));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.java
deleted file mode 100644
index e994760..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountFileSpout.java
+++ /dev/null
@@ -1,39 +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.wordcount.stormoperators;
-
-import org.apache.flink.stormcompatibility.util.StormFileSpout;
-
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-
-/**
- * Implements a Storm Spout that reads data from a given local file.
- */
-public final class StormWordCountFileSpout extends StormFileSpout {
-	private static final long serialVersionUID = 2372251989250954503L;
-
-	public StormWordCountFileSpout(String path) {
-		super(path);
-	}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields("sentence"));
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.java
deleted file mode 100644
index 372f66f..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/StormWordCountInMemorySpout.java
+++ /dev/null
@@ -1,40 +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.wordcount.stormoperators;
-
-import org.apache.flink.examples.java.wordcount.util.WordCountData;
-import org.apache.flink.stormcompatibility.util.StormInMemorySpout;
-
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-
-/**
- * Implements a Storm Spout that reads data from {@link WordCountData#WORDS}.
- */
-public final class StormWordCountInMemorySpout extends StormInMemorySpout<String> {
-	private static final long serialVersionUID = 8832143302409465843L;
-
-	public StormWordCountInMemorySpout() {
-		super(WordCountData.WORDS);
-	}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields("sentence"));
-	}
-}


[07/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataPojos.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataPojos.java
deleted file mode 100644
index f965a28..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataPojos.java
+++ /dev/null
@@ -1,59 +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.wordcount.stormoperators;
-
-import java.io.Serializable;
-
-import org.apache.flink.examples.java.wordcount.util.WordCountData;
-
-public class WordCountDataPojos {
-	public static Sentence[] SENTENCES;
-
-	static {
-		SENTENCES = new Sentence[WordCountData.WORDS.length];
-		for (int i = 0; i < SENTENCES.length; ++i) {
-			SENTENCES[i] = new Sentence(WordCountData.WORDS[i]);
-		}
-	}
-
-	public static class Sentence implements Serializable {
-		private static final long serialVersionUID = -7336372859203407522L;
-
-		private String sentence;
-
-		public Sentence() {
-		}
-
-		public Sentence(String sentence) {
-			this.sentence = sentence;
-		}
-
-		public String getSentence() {
-			return sentence;
-		}
-
-		public void setSentence(String sentence) {
-			this.sentence = sentence;
-		}
-
-		@Override
-		public String toString() {
-			return "(" + this.sentence + ")";
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataTuple.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataTuple.java
deleted file mode 100644
index 732f0ae..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/wordcount/stormoperators/WordCountDataTuple.java
+++ /dev/null
@@ -1,34 +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.wordcount.stormoperators;
-
-import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.examples.java.wordcount.util.WordCountData;
-
-@SuppressWarnings("unchecked")
-public class WordCountDataTuple {
-	public static Tuple1<String>[] TUPLES;
-
-	static {
-		TUPLES = new Tuple1[WordCountData.WORDS.length];
-		for (int i = 0; i < TUPLES.length; ++i) {
-			TUPLES[i] = new Tuple1<String>(WordCountData.WORDS[i]);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.java
deleted file mode 100644
index dd6d0d9..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/api/StormTestBase.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.api;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.StreamingMode;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.test.util.AbstractTestBase;
-
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-
-/**
- * Base class for Storm tests.
- */
-public abstract class StormTestBase extends AbstractTestBase {
-	
-	public static final int DEFAULT_PARALLELISM = 4;
-	
-	public StormTestBase() {
-		this(new Configuration());
-	}
-	
-	public StormTestBase(Configuration config) {
-		super(config, StreamingMode.STREAMING);
-		setTaskManagerNumSlots(DEFAULT_PARALLELISM);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Methods to create the test program and for pre- and post- test work
-	// ------------------------------------------------------------------------
-
-	protected abstract void testProgram() throws Exception;
-
-	protected void preSubmit() throws Exception {}
-
-	protected void postSubmit() throws Exception {}
-
-	// ------------------------------------------------------------------------
-	//  Test entry point
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testJob() throws Exception {
-		try {
-			// pre-submit
-			try {
-				preSubmit();
-			}
-			catch (Exception e) {
-				System.err.println(e.getMessage());
-				e.printStackTrace();
-				fail("Pre-submit work caused an error: " + e.getMessage());
-			}
-
-			// prepare the test environment
-			startCluster();
-
-			// we need to initialize the stream test environment, and the storm local cluster
-			TestStreamEnvironment.setAsContext(this.executor, DEFAULT_PARALLELISM);
-			
-			FlinkLocalCluster.initialize(new FlinkLocalCluster.LocalClusterFactory() {
-				@Override
-				public FlinkLocalCluster createLocalCluster() {
-					return new FlinkLocalCluster(executor);
-				}
-			});
-
-			// call the test program
-			try {
-				testProgram();
-			}
-			catch (Exception e) {
-				System.err.println(e.getMessage());
-				e.printStackTrace();
-				fail("Error while calling the test program: " + e.getMessage());
-			}
-
-			// post-submit
-			try {
-				postSubmit();
-			}
-			catch (Exception e) {
-				System.err.println(e.getMessage());
-				e.printStackTrace();
-				fail("Post-submit work caused an error: " + e.getMessage());
-			}
-		}
-		finally {
-			// reset the FlinkLocalCluster to its default behavior
-			FlinkLocalCluster.initialize(new FlinkLocalCluster.DefaultLocalClusterFactory());
-			
-			// reset the StreamExecutionEnvironment to its default behavior
-			TestStreamEnvironment.unsetAsContext();
-			
-			// clean up all resources
-			stopCluster();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java
deleted file mode 100644
index a858f36..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormBoltITCase.java
+++ /dev/null
@@ -1,49 +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.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.test.testdata.WordCountData;
-
-public class ExclamationWithStormBoltITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-	protected String exclamationNum;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-		this.exclamationNum = "3";
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(ExclamationData.TEXT_WITH_EXCLAMATIONS, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		ExclamationWithStormBolt.main(new String[]{this.textPath, this.resultPath, this.exclamationNum});
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java
deleted file mode 100644
index 2a8ac24..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/ExclamationWithStormSpoutITCase.java
+++ /dev/null
@@ -1,46 +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.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.test.testdata.WordCountData;
-
-public class ExclamationWithStormSpoutITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(ExclamationData.TEXT_WITH_EXCLAMATIONS, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		ExclamationWithStormSpout.main(new String[]{this.textPath, this.resultPath});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java
deleted file mode 100644
index a19f3af..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/StormExclamationLocalITCase.java
+++ /dev/null
@@ -1,48 +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.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.test.testdata.WordCountData;
-
-public class StormExclamationLocalITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-	protected String exclamationNum;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-		this.exclamationNum = "3";
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(ExclamationData.TEXT_WITH_EXCLAMATIONS, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		StormExclamationLocal.main(new String[]{this.textPath, this.resultPath, this.exclamationNum});
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/util/ExclamationData.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/util/ExclamationData.java
deleted file mode 100644
index 8b823b5..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/exclamation/util/ExclamationData.java
+++ /dev/null
@@ -1,98 +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.exclamation.util;
-
-public class ExclamationData {
-
-	public static final String TEXT_WITH_EXCLAMATIONS =
-			"Goethe - Faust: Der Tragoedie erster Teil!!!!!!\n"
-					+ "Prolog im Himmel.!!!!!!\n"
-					+ "Der Herr. Die himmlischen Heerscharen. Nachher Mephistopheles. Die drei!!!!!!\n"
-					+ "Erzengel treten vor.!!!!!!\n"
-					+ "RAPHAEL: Die Sonne toent, nach alter Weise, In Brudersphaeren Wettgesang,!!!!!!\n"
-					+ "Und ihre vorgeschriebne Reise Vollendet sie mit Donnergang. Ihr Anblick!!!!!!\n"
-					+ "gibt den Engeln Staerke, Wenn keiner Sie ergruenden mag; die unbegreiflich!!!!!!\n"
-					+ "hohen Werke Sind herrlich wie am ersten Tag.!!!!!!\n"
-					+ "GABRIEL: Und schnell und unbegreiflich schnelle Dreht sich umher der Erde!!!!!!\n"
-					+ "Pracht; Es wechselt Paradieseshelle Mit tiefer, schauervoller Nacht. Es!!!!!!\n"
-					+ "schaeumt das Meer in breiten Fluessen Am tiefen Grund der Felsen auf, Und!!!!!!\n"
-					+ "Fels und Meer wird fortgerissen Im ewig schnellem Sphaerenlauf.!!!!!!\n"
-					+ "MICHAEL: Und Stuerme brausen um die Wette Vom Meer aufs Land, vom Land!!!!!!\n"
-					+ "aufs Meer, und bilden wuetend eine Kette Der tiefsten Wirkung rings umher.!!!!!!\n"
-					+ "Da flammt ein blitzendes Verheeren Dem Pfade vor des Donnerschlags. Doch!!!!!!\n"
-					+ "deine Boten, Herr, verehren Das sanfte Wandeln deines Tags.!!!!!!\n"
-					+ "ZU DREI: Der Anblick gibt den Engeln Staerke, Da keiner dich ergruenden!!!!!!\n"
-					+ "mag, Und alle deine hohen Werke Sind herrlich wie am ersten Tag.!!!!!!\n"
-					+ "MEPHISTOPHELES: Da du, o Herr, dich einmal wieder nahst Und fragst, wie!!!!!!\n"
-					+ "alles sich bei uns befinde, Und du mich sonst gewoehnlich gerne sahst, So!!!!!!\n"
-					+ "siehst du mich auch unter dem Gesinde. Verzeih, ich kann nicht hohe Worte!!!!!!\n"
-					+ "machen, Und wenn mich auch der ganze Kreis verhoehnt; Mein Pathos braechte!!!!!!\n"
-					+ "dich gewiss zum Lachen, Haettst du dir nicht das Lachen abgewoehnt. Von!!!!!!\n"
-					+ "Sonn' und Welten weiss ich nichts zu sagen, Ich sehe nur, wie sich die!!!!!!\n"
-					+ "Menschen plagen. Der kleine Gott der Welt bleibt stets von gleichem!!!!!!\n"
-					+ "Schlag, Und ist so wunderlich als wie am ersten Tag. Ein wenig besser!!!!!!\n"
-					+ "wuerd er leben, Haettst du ihm nicht den Schein des Himmelslichts gegeben;!!!!!!\n"
-					+ "Er nennt's Vernunft und braucht's allein, Nur tierischer als jedes Tier!!!!!!\n"
-					+ "zu sein. Er scheint mir, mit Verlaub von euer Gnaden, Wie eine der!!!!!!\n"
-					+ "langbeinigen Zikaden, Die immer fliegt und fliegend springt Und gleich im!!!!!!\n"
-					+ "Gras ihr altes Liedchen singt; Und laeg er nur noch immer in dem Grase! In!!!!!!\n"
-					+ "jeden Quark begraebt er seine Nase.!!!!!!\n"
-					+ "DER HERR: Hast du mir weiter nichts zu sagen? Kommst du nur immer!!!!!!\n"
-					+ "anzuklagen? Ist auf der Erde ewig dir nichts recht?!!!!!!\n"
-					+ "MEPHISTOPHELES: Nein Herr! ich find es dort, wie immer, herzlich!!!!!!\n"
-					+ "schlecht. Die Menschen dauern mich in ihren Jammertagen, Ich mag sogar!!!!!!\n"
-					+ "die armen selbst nicht plagen.!!!!!!\n" + "DER HERR: Kennst du den Faust?!!!!!!\n"
-					+ "MEPHISTOPHELES: Den Doktor?!!!!!!\n"
-					+ "DER HERR: Meinen Knecht!!!!!!!\n"
-					+ "MEPHISTOPHELES: Fuerwahr! er dient Euch auf besondre Weise. Nicht irdisch!!!!!!\n"
-					+ "ist des Toren Trank noch Speise. Ihn treibt die Gaerung in die Ferne, Er!!!!!!\n"
-					+ "ist sich seiner Tollheit halb bewusst; Vom Himmel fordert er die schoensten!!!!!!\n"
-					+ "Sterne Und von der Erde jede hoechste Lust, Und alle Naeh und alle Ferne!!!!!!\n"
-					+ "Befriedigt nicht die tiefbewegte Brust.!!!!!!\n"
-					+ "DER HERR: Wenn er mir auch nur verworren dient, So werd ich ihn bald in!!!!!!\n"
-					+ "die Klarheit fuehren. Weiss doch der Gaertner, wenn das Baeumchen gruent, Das!!!!!!\n"
-					+ "Bluet und Frucht die kuenft'gen Jahre zieren.!!!!!!\n"
-					+ "MEPHISTOPHELES: Was wettet Ihr? den sollt Ihr noch verlieren! Wenn Ihr!!!!!!\n"
-					+ "mir die Erlaubnis gebt, Ihn meine Strasse sacht zu fuehren.!!!!!!\n"
-					+ "DER HERR: Solang er auf der Erde lebt, So lange sei dir's nicht verboten,!!!!!!\n"
-					+ "Es irrt der Mensch so lang er strebt.!!!!!!\n"
-					+ "MEPHISTOPHELES: Da dank ich Euch; denn mit den Toten Hab ich mich niemals!!!!!!\n"
-					+ "gern befangen. Am meisten lieb ich mir die vollen, frischen Wangen. Fuer!!!!!!\n"
-					+ "einem Leichnam bin ich nicht zu Haus; Mir geht es wie der Katze mit der Maus.!!!!!!\n"
-					+ "DER HERR: Nun gut, es sei dir ueberlassen! Zieh diesen Geist von seinem!!!!!!\n"
-					+ "Urquell ab, Und fuehr ihn, kannst du ihn erfassen, Auf deinem Wege mit!!!!!!\n"
-					+ "herab, Und steh beschaemt, wenn du bekennen musst: Ein guter Mensch, in!!!!!!\n"
-					+ "seinem dunklen Drange, Ist sich des rechten Weges wohl bewusst.!!!!!!\n"
-					+ "MEPHISTOPHELES: Schon gut! nur dauert es nicht lange. Mir ist fuer meine!!!!!!\n"
-					+ "Wette gar nicht bange. Wenn ich zu meinem Zweck gelange, Erlaubt Ihr mir!!!!!!\n"
-					+ "Triumph aus voller Brust. Staub soll er fressen, und mit Lust, Wie meine!!!!!!\n"
-					+ "Muhme, die beruehmte Schlange.!!!!!!\n"
-					+ "DER HERR: Du darfst auch da nur frei erscheinen; Ich habe deinesgleichen!!!!!!\n"
-					+ "nie gehasst. Von allen Geistern, die verneinen, ist mir der Schalk am!!!!!!\n"
-					+ "wenigsten zur Last. Des Menschen Taetigkeit kann allzu leicht erschlaffen,!!!!!!\n"
-					+ "er liebt sich bald die unbedingte Ruh; Drum geb ich gern ihm den Gesellen!!!!!!\n"
-					+ "zu, Der reizt und wirkt und muss als Teufel schaffen. Doch ihr, die echten!!!!!!\n"
-					+ "Goettersoehne, Erfreut euch der lebendig reichen Schoene! Das Werdende, das!!!!!!\n"
-					+ "ewig wirkt und lebt, Umfass euch mit der Liebe holden Schranken, Und was!!!!!!\n"
-					+ "in schwankender Erscheinung schwebt, Befestigt mit dauernden Gedanken!!!!!!!\n"
-					+ "(Der Himmel schliesst, die Erzengel verteilen sich.)!!!!!!\n"
-					+ "MEPHISTOPHELES (allein): Von Zeit zu Zeit seh ich den Alten gern, Und!!!!!!\n"
-					+ "huete mich, mit ihm zu brechen. Es ist gar huebsch von einem grossen Herrn,!!!!!!\n"
-					+ "So menschlich mit dem Teufel selbst zu sprechen.!!!!!!";
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java
deleted file mode 100644
index 305245b..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/BoltSplitITCase.java
+++ /dev/null
@@ -1,28 +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.split;
-
-import org.junit.Test;
-
-public class BoltSplitITCase {
-
-	@Test
-	public void testTopology() throws Exception {
-		StormSplitStreamBoltLocal.main(new String[] { "0", "/dev/null" });
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java
deleted file mode 100644
index c40e054..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBolt.java
+++ /dev/null
@@ -1,61 +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.split;
-
-import java.util.Map;
-
-import backtype.storm.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.topology.base.BaseRichBolt;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Tuple;
-import backtype.storm.tuple.Values;
-
-public class SplitBolt extends BaseRichBolt {
-	private static final long serialVersionUID = -6627606934204267173L;
-
-	public static final String EVEN_STREAM = "even";
-	public static final String ODD_STREAM = "odd";
-
-	private OutputCollector collector;
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
-		this.collector = collector;
-
-	}
-
-	@Override
-	public void execute(Tuple input) {
-		if (input.getInteger(0) % 2 == 0) {
-			this.collector.emit(EVEN_STREAM, new Values(input.getInteger(0)));
-		} else {
-			this.collector.emit(ODD_STREAM, new Values(input.getInteger(0)));
-		}
-	}
-
-	@Override
-	public void declareOutputFields(OutputFieldsDeclarer declarer) {
-		Fields schema = new Fields("number");
-		declarer.declareStream(EVEN_STREAM, schema);
-		declarer.declareStream(ODD_STREAM, schema);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 5f637d3..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java
+++ /dev/null
@@ -1,87 +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.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;
-
-public class SplitBoltTopology {
-	public final static String spoutId = "randomSource";
-	public final static String boltId = "splitBolt";
-	public final static String evenVerifierId = "evenVerifier";
-	public final static String oddVerifierId = "oddVerifier";
-	public final static String sinkId = "sink";
-	private final static OutputFormatter formatter = new TupleOutputFormatter();
-
-	public static FlinkTopologyBuilder buildTopology() {
-		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
-
-		builder.setSpout(spoutId, new RandomSpout(false, seed));
-		builder.setBolt(boltId, new SplitBolt()).shuffleGrouping(spoutId);
-		builder.setBolt(evenVerifierId, new VerifyAndEnrichBolt(true)).shuffleGrouping(boltId,
-				SplitBolt.EVEN_STREAM);
-		builder.setBolt(oddVerifierId, new VerifyAndEnrichBolt(false)).shuffleGrouping(boltId,
-				SplitBolt.ODD_STREAM);
-
-		// emit result
-		if (outputPath != null) {
-			// 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);
-		} else {
-			builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4)
-					.shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId);
-		}
-
-		return builder;
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static long seed = System.currentTimeMillis();
-	private static String outputPath = null;
-
-	static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			if (args.length == 2) {
-				seed = Long.parseLong(args[0]);
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: SplitBoltTopology <seed> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing SplitBoltTopology example with random data");
-			System.out.println("  Usage: SplitBoltTopology <seed> <result path>");
-		}
-
-		return true;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java
deleted file mode 100644
index 613fd10..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitSpoutTopology.java
+++ /dev/null
@@ -1,85 +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.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;
-
-public class SplitSpoutTopology {
-	public final static String spoutId = "randomSplitSource";
-	public final static String evenVerifierId = "evenVerifier";
-	public final static String oddVerifierId = "oddVerifier";
-	public final static String sinkId = "sink";
-	private final static OutputFormatter formatter = new TupleOutputFormatter();
-
-	public static FlinkTopologyBuilder buildTopology() {
-		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
-
-		builder.setSpout(spoutId, new RandomSpout(true, seed));
-		builder.setBolt(evenVerifierId, new VerifyAndEnrichBolt(true)).shuffleGrouping(spoutId,
-				RandomSpout.EVEN_STREAM);
-		builder.setBolt(oddVerifierId, new VerifyAndEnrichBolt(false)).shuffleGrouping(spoutId,
-				RandomSpout.ODD_STREAM);
-
-		// emit result
-		if (outputPath != null) {
-			// 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);
-		} else {
-			builder.setBolt(sinkId, new StormBoltPrintSink(formatter), 4)
-			.shuffleGrouping(evenVerifierId).shuffleGrouping(oddVerifierId);
-		}
-
-		return builder;
-	}
-
-	// *************************************************************************
-	// UTIL METHODS
-	// *************************************************************************
-
-	private static long seed = System.currentTimeMillis();
-	private static String outputPath = null;
-
-	static boolean parseParameters(final String[] args) {
-
-		if (args.length > 0) {
-			// parse input arguments
-			if (args.length == 2) {
-				seed = Long.parseLong(args[0]);
-				outputPath = args[1];
-			} else {
-				System.err.println("Usage: SplitSpoutTopology <seed> <result path>");
-				return false;
-			}
-		} else {
-			System.out.println("Executing SplitSpoutTopology example with random data");
-			System.out.println("  Usage: SplitSpoutTopology <seed> <result path>");
-		}
-
-		return true;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java
deleted file mode 100644
index f30e160..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SpoutSplitITCase.java
+++ /dev/null
@@ -1,28 +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.split;
-
-import org.junit.Test;
-
-public class SpoutSplitITCase {
-
-	@Test
-	public void testTopology() throws Exception {
-		StormSplitStreamSpoutLocal.main(new String[] { "0", "/dev/null" });
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java
deleted file mode 100644
index 028f6d1..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamBoltLocal.java
+++ /dev/null
@@ -1,51 +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.split;
-
-import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
-import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder;
-
-import backtype.storm.utils.Utils;
-
-public class StormSplitStreamBoltLocal {
-	public final static String topologyId = "Bolt split stream example";
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!SplitBoltTopology.parseParameters(args)) {
-			return;
-		}
-
-		// build Topology the Storm way
-		final FlinkTopologyBuilder builder = SplitBoltTopology.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();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java
deleted file mode 100644
index cc5acd9..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/StormSplitStreamSpoutLocal.java
+++ /dev/null
@@ -1,51 +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.split;
-
-import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
-import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder;
-
-import backtype.storm.utils.Utils;
-
-public class StormSplitStreamSpoutLocal {
-	public final static String topologyId = "Spout split stream example";
-
-	// *************************************************************************
-	// PROGRAM
-	// *************************************************************************
-
-	public static void main(final String[] args) throws Exception {
-
-		if (!SplitSpoutTopology.parseParameters(args)) {
-			return;
-		}
-
-		// build Topology the Storm way
-		final FlinkTopologyBuilder builder = SplitSpoutTopology.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();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
deleted file mode 100644
index c9516ff..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
+++ /dev/null
@@ -1,45 +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.wordcount;
-
-import org.apache.flink.stormcompatibility.api.StormTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class BoltTokenizerWordCountITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		BoltTokenizerWordCount.main(new String[]{this.textPath, this.resultPath});
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java
deleted file mode 100644
index 351014e..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountPojoITCase.java
+++ /dev/null
@@ -1,45 +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.wordcount;
-
-import org.apache.flink.stormcompatibility.api.StormTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class BoltTokenizerWordCountPojoITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		BoltTokenizerWordCountPojo.main(new String[]{this.textPath, this.resultPath});
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java
deleted file mode 100644
index c2ed088..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountWithNamesITCase.java
+++ /dev/null
@@ -1,45 +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.wordcount;
-
-import org.apache.flink.stormcompatibility.api.StormTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class BoltTokenizerWordCountWithNamesITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		BoltTokenizerWordCountWithNames.main(new String[]{this.textPath, this.resultPath});
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
deleted file mode 100644
index 93361c5..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
+++ /dev/null
@@ -1,45 +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.wordcount;
-
-import org.apache.flink.stormcompatibility.api.StormTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class SpoutSourceWordCountITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		SpoutSourceWordCount.main(new String[]{this.textPath, this.resultPath});
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
deleted file mode 100644
index 6b51cbd..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
+++ /dev/null
@@ -1,45 +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.wordcount;
-
-import org.apache.flink.stormcompatibility.api.StormTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class StormWordCountLocalITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		StormWordCountLocal.main(new String[]{this.textPath, this.resultPath});
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java
deleted file mode 100644
index a9e9884..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalNamedITCase.java
+++ /dev/null
@@ -1,45 +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.wordcount;
-
-import org.apache.flink.stormcompatibility.api.StormTestBase;
-import org.apache.flink.test.testdata.WordCountData;
-
-public class StormWordCountLocalNamedITCase extends StormTestBase {
-
-	protected String textPath;
-	protected String resultPath;
-
-	@Override
-	protected void preSubmit() throws Exception {
-		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
-		this.resultPath = this.getTempDirPath("result");
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		StormWordCountNamedLocal.main(new String[] { this.textPath, this.resultPath });
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j-test.properties b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j-test.properties
deleted file mode 100644
index 0b686e5..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j-test.properties
+++ /dev/null
@@ -1,27 +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.
-################################################################################
-
-# Set root logger level to DEBUG and its only appender to A1.
-log4j.rootLogger=OFF, A1
-
-# A1 is set to be a ConsoleAppender.
-log4j.appender.A1=org.apache.log4j.ConsoleAppender
-
-# A1 uses PatternLayout.
-log4j.appender.A1.layout=org.apache.log4j.PatternLayout
-log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j.properties b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j.properties
deleted file mode 100644
index ed2bbcb..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,27 +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.
-################################################################################
-
-# This file ensures that tests executed from the IDE show log output
-
-log4j.rootLogger=OFF, console
-
-# Log all infos in the given file
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target = System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/logback-test.xml b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/logback-test.xml
deleted file mode 100644
index 4f56748..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,30 +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.
-  -->
-
-<configuration>
-    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n</pattern>
-        </encoder>
-    </appender>
-
-    <root level="WARN">
-        <appender-ref ref="STDOUT"/>
-    </root>
-    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-compatibility/pom.xml b/flink-contrib/flink-storm-compatibility/pom.xml
deleted file mode 100644
index 803336f..0000000
--- a/flink-contrib/flink-storm-compatibility/pom.xml
+++ /dev/null
@@ -1,40 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-contrib-parent</artifactId>
-		<version>0.10-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-storm-compatibility-parent</artifactId>
-	<name>flink-storm-compatibility</name>
-	<packaging>pom</packaging>
-
-	<modules>
-		<module>flink-storm-compatibility-core</module>
-		<module>flink-storm-compatibility-examples</module>
-	</modules>
-</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/README.md
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/README.md b/flink-contrib/flink-storm-examples/README.md
new file mode 100644
index 0000000..c3247f6
--- /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.

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/pom.xml
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/pom.xml b/flink-contrib/flink-storm-examples/pom.xml
new file mode 100644
index 0000000..de6c0cb
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/pom.xml
@@ -0,0 +1,364 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-contrib-parent</artifactId>
+		<version>0.10-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-storm-examples</artifactId>
+	<name>flink-storm-examples</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-storm</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-java-examples</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-core</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<!-- get default data from flink-java-examples package -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-dependency-plugin</artifactId>
+				<version>2.9</version><!--$NO-MVN-MAN-VER$-->
+				<executions>
+					<execution>
+						<id>unpack</id>
+						<phase>prepare-package</phase>
+						<goals>
+							<goal>unpack</goal>
+						</goals>
+						<configuration>
+							<artifactItems>
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-java-examples</artifactId>
+									<version>${project.version}</version>
+									<type>jar</type>
+									<overWrite>false</overWrite>
+									<outputDirectory>${project.build.directory}/classes</outputDirectory>
+									<includes>org/apache/flink/examples/java/wordcount/util/WordCountData.class</includes>
+								</artifactItem>
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-storm</artifactId>
+									<version>${project.version}</version>
+									<type>jar</type>
+									<overWrite>false</overWrite>
+									<outputDirectory>${project.build.directory}/classes</outputDirectory>
+								</artifactItem>
+								<artifactItem>
+									<groupId>org.apache.storm</groupId>
+									<artifactId>storm-core</artifactId>
+									<version>0.9.4</version>
+									<type>jar</type>
+									<overWrite>false</overWrite>
+									<outputDirectory>${project.build.directory}/classes</outputDirectory>
+									<!-- need to exclude to be able to run
+									       * StormWordCountRemoteByClient and
+									       * StormWordCountRemoteBySubmitter
+									     within Eclipse -->
+									<excludes>defaults.yaml</excludes>
+								</artifactItem>
+								<artifactItem>
+									<groupId>com.googlecode.json-simple</groupId>
+									<artifactId>json-simple</artifactId>
+									<version>1.1</version>
+									<type>jar</type>
+									<overWrite>false</overWrite>
+									<outputDirectory>${project.build.directory}/classes</outputDirectory>
+								</artifactItem>
+								<artifactItem>
+									<groupId>org.yaml</groupId>
+									<artifactId>snakeyaml</artifactId>
+									<version>1.11</version>
+									<type>jar</type>
+									<overWrite>false</overWrite>
+									<outputDirectory>${project.build.directory}/classes</outputDirectory>
+								</artifactItem>
+							</artifactItems>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+
+			<!-- self-contained jars for each example -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+
+				<executions>
+
+					<!-- WordCount Spout source-->
+					<!-- example for embedded spout - for whole topologies see "WordCount Storm topology" example below -->
+					<execution>
+						<id>WordCount-SpoutSource</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<finalName>WordCount</finalName>
+							<classifier>SpoutSource</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.storm.wordcount.SpoutSourceWordCount</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<!-- from storm-core -->
+								<include>backtype/storm/topology/*.class</include>
+								<include>backtype/storm/spout/*.class</include>
+								<include>backtype/storm/task/*.class</include>
+								<include>backtype/storm/tuple/*.class</include>
+								<include>backtype/storm/generated/*.class</include>
+								<include>backtype/storm/metric/**/*.class</include>
+								<include>org/apache/thrift7/**/*.class</include>
+								<!-- Storm's recursive dependencies -->
+								<include>org/json/simple/**/*.class</include>
+								<!-- compatibility layer -->
+								<include>org/apache/flink/storm/api/*.class</include>
+								<include>org/apache/flink/storm/util/*.class</include>
+								<include>org/apache/flink/storm/wrappers/*.class</include>
+								<!-- Word Count -->
+								<include>org/apache/flink/storm/wordcount/SpoutSourceWordCount.class</include>
+								<include>org/apache/flink/storm/wordcount/SpoutSourceWordCount$*.class</include>
+								<include>org/apache/flink/storm/wordcount/operators/WordCountFileSpout.class</include>
+								<include>org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.class</include>
+								<include>org/apache/flink/storm/util/AbstractLineSpout.class</include>
+								<include>org/apache/flink/storm/util/FileSpout.class</include>
+								<include>org/apache/flink/storm/util/InMemorySpout.class</include>
+								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- WordCount Bolt tokenizer-->
+					<!-- example for embedded bolt - for whole topologies see "WordCount Storm topology" example below -->
+					<execution>
+						<id>WordCount-BoltTokenizer</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<finalName>WordCount</finalName>
+							<classifier>BoltTokenizer</classifier>
+
+							<archive>
+								<manifestEntries>
+									<program-class>org.apache.flink.storm.wordcount.BoltTokenizerWordCount</program-class>
+								</manifestEntries>
+							</archive>
+
+							<includes>
+								<!-- from storm-core -->
+								<include>backtype/storm/topology/*.class</include>
+								<include>backtype/storm/spout/*.class</include>
+								<include>backtype/storm/task/*.class</include>
+								<include>backtype/storm/tuple/*.class</include>
+								<include>backtype/storm/generated/*.class</include>
+								<include>backtype/storm/metric/**/*.class</include>
+								<include>org/apache/thrift7/**/*.class</include>
+								<!-- Storm's recursive dependencies -->
+								<include>org/json/simple/**/*.class</include>
+								<!-- compatibility layer -->
+								<include>org/apache/flink/storm/api/*.class</include>
+								<include>org/apache/flink/storm/util/*.class</include>
+								<include>org/apache/flink/storm/wrappers/*.class</include>
+								<!-- Word Count -->
+								<include>org/apache/flink/storm/wordcount/BoltTokenizerWordCount.class</include>
+								<include>org/apache/flink/storm/wordcount/operators/BoltTokenizer.class</include>
+								<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
+							</includes>
+						</configuration>
+					</execution>
+
+					<!-- WordCount Storm topology-->
+					<!-- Example for whole topologies (ie, if FlinkTopologyBuilder is used) -->
+					<!-- We cannot use maven-jar-plugin because 'defaults.yaml' must be included in jar.
+					     However, we excluded 'defaults.yaml' in dependency-plugin to get clean Eclipse environment.
+					     Thus, 'defaults.yaml' is not available for maven-jar-plugin.
+					     Nevertheless, we register an empty jar with corresponding name, such that the final jar can be installed to local maven repository.
+					     We use maven-shade-plugin to build the actual jar (which will replace the empty jar). -->
+					<execution>
+						<id>WordCount-StormTopology</id>
+						<phase>package</phase>
+						<goals>
+							<goal>jar</goal>
+						</goals>
+						<configuration>
+							<finalName>WordCount</finalName>
+							<classifier>StormTopology</classifier>
+						</configuration>
+					</execution>
+
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
+			<!-- WordCount Storm topology-->
+			<!-- Cannot use maven-jar-plugin because 'defaults.yaml' must be included in jar -->
+			<!-- Build StormTopolgy jar to overwrite empty jar created with maven-jar-plugin. -->
+			<plugin>
+				<artifactId>maven-shade-plugin</artifactId>
+				<groupId>org.apache.maven.plugins</groupId>
+				<executions>
+					<execution>
+						<id>WordCount-StormTopology</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<finalName>WordCount-StormTopology</finalName>
+
+							<artifactSet>
+								<includes>
+									<include>org.apache.storm:storm-core</include>
+									<!-- Storm's recursive dependencies -->
+									<include>org.yaml:snakeyaml</include>
+									<include>com.googlecode.json-simple:json-simple</include>
+									<include>org.apache.flink:flink-storm</include>
+									<include>org.apache.flink:flink-storm-examples</include>
+								</includes>
+							</artifactSet>
+							<filters>
+								<filter>
+									<artifact>org.apache.storm:storm-core</artifact>
+									<includes>
+										<include>defaults.yaml</include>
+										<include>backtype/storm/*.class</include>
+										<include>backtype/storm/topology/*.class</include>
+										<include>backtype/storm/spout/*.class</include>
+										<include>backtype/storm/task/*.class</include>
+										<include>backtype/storm/tuple/*.class</include>
+										<include>backtype/storm/generated/*.class</include>
+										<include>backtype/storm/metric/**/*.class</include>
+										<include>backtype/storm/utils/*.class</include>
+										<include>backtype/storm/serialization/*.class</include>
+										<include>org/apache/storm/curator/**/*.class</include>
+										<include>org/apache/thrift7/**/*.class</include>
+										<!-- Storm's recursive dependencies -->
+										<include>org/json/simple/**/*.class</include>
+										<include>org/yaml/snakeyaml/**/*.class</include>
+									</includes>
+								</filter>
+								<filter>
+									<artifact>org.apache.flink:flink-storm-examples</artifact>
+									<includes>
+										<include>org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.class</include>
+										<include>org/apache/flink/storm/wordcount/WordCountTopology.class</include>
+										<include>org/apache/flink/storm/wordcount/operators/*.class</include>
+										<include>org/apache/flink/storm/util/*.class</include>
+										<include>org/apache/flink/examples/java/wordcount/util/WordCountData.class</include>
+									</includes>
+								</filter>
+								<filter>
+									<artifact>org.apache.flink:flink-storm</artifact>
+									<includes>
+										<include>org/apache/flink/storm/api/*.class</include>
+										<include>org/apache/flink/storm/util/*.class</include>
+										<include>org/apache/flink/storm/wrappers/*.class</include>
+									</includes>
+								</filter>
+							</filters>
+							<transformers>
+								<transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+									<mainClass>org.apache.flink.storm.wordcount.WordCountRemoteBySubmitter</mainClass>
+								</transformer>
+							</transformers>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+
+		<pluginManagement>
+			<plugins>
+				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+				<plugin>
+					<groupId>org.eclipse.m2e</groupId>
+					<artifactId>lifecycle-mapping</artifactId>
+					<version>1.0.0</version>
+					<configuration>
+						<lifecycleMappingMetadata>
+							<pluginExecutions>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-dependency-plugin</artifactId>
+										<versionRange>[2.9,)</versionRange>
+										<goals>
+											<goal>unpack</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+							</pluginExecutions>
+						</lifecycleMappingMetadata>
+					</configuration>
+				</plugin>
+			</plugins>
+		</pluginManagement>
+
+	</build>
+
+</project>


[14/15] flink git commit: [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 cle

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java
new file mode 100644
index 0000000..1891873
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FiniteTestSpout.java
@@ -0,0 +1,77 @@
+/*
+ * 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 backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+
+import java.util.Map;
+
+public class FiniteTestSpout implements IRichSpout {
+	private static final long serialVersionUID = 7992419478267824279L;
+
+	private int numberOfOutputTuples;
+	private SpoutOutputCollector collector;
+
+	public FiniteTestSpout(final int numberOfOutputTuples) {
+		this.numberOfOutputTuples = numberOfOutputTuples;
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void close() {/* nothing to do */}
+
+	@Override
+	public void activate() {/* nothing to do */}
+
+	@Override
+	public void deactivate() {/* nothing to do */}
+
+	@Override
+	public void nextTuple() {
+		if (--this.numberOfOutputTuples >= 0) {
+			this.collector.emit(new Values(new Integer(this.numberOfOutputTuples)));
+		}
+	}
+
+	@Override
+	public void ack(final Object msgId) {/* nothing to do */}
+
+	@Override
+	public void fail(final Object msgId) {/* nothing to do */}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields("dummy"));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java
new file mode 100644
index 0000000..8e63563
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarerTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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 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.junit.Assert;
+import org.junit.Test;
+
+import java.util.LinkedList;
+
+public class FlinkOutputFieldsDeclarerTest extends AbstractTest {
+
+
+
+	@Test
+	public void testNull() {
+		Assert.assertNull(new FlinkOutputFieldsDeclarer().getOutputType(null));
+	}
+
+	@Test
+	public void testDeclare() {
+		for (int i = 0; i < 2; ++i) { // test case: simple / non-direct
+			for (int j = 1; j < 2; ++j) { // number of streams
+				for (int k = 0; k <= 25; ++k) { // number of attributes
+					this.runDeclareTest(i, j, k);
+				}
+			}
+		}
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareSimpleToManyAttributes() {
+		this.runDeclareTest(0, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareNonDirectToManyAttributes() {
+		this.runDeclareTest(1, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareDefaultStreamToManyAttributes() {
+		this.runDeclareTest(2, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareFullToManyAttributes() {
+		this.runDeclareTest(3, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	private void runDeclareTest(final int testCase, final int numberOfStreams,
+			final int numberOfAttributes) {
+		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+
+		String[] streams = null;
+		if (numberOfStreams > 1 || r.nextBoolean()) {
+			streams = new String[numberOfStreams];
+			for (int i = 0; i < numberOfStreams; ++i) {
+				streams[i] = "stream" + i;
+			}
+		}
+
+		final String[] attributes = new String[numberOfAttributes];
+		for (int i = 0; i < attributes.length; ++i) {
+			attributes[i] = "a" + i;
+		}
+
+		switch (testCase) {
+		case 0:
+			this.declareSimple(declarer, streams, attributes);
+			break;
+		default:
+			this.declareNonDirect(declarer, streams, attributes);
+		}
+
+		if (streams == null) {
+			streams = new String[] { Utils.DEFAULT_STREAM_ID };
+		}
+
+		for (String stream : streams) {
+			final TypeInformation<?> type = declarer.getOutputType(stream);
+
+			if (numberOfAttributes == 1) {
+				Assert.assertEquals(type.getClass(), GenericTypeInfo.class);
+				Assert.assertEquals(type.getTypeClass(), Object.class);
+			} else {
+				Assert.assertEquals(numberOfAttributes, type.getArity());
+				Assert.assertTrue(type.isTupleType());
+			}
+		}
+	}
+
+	private void declareSimple(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
+			final String[] attributes) {
+
+		if (streams != null) {
+			for (String stream : streams) {
+				declarer.declareStream(stream, new Fields(attributes));
+			}
+		} else {
+			declarer.declare(new Fields(attributes));
+		}
+	}
+
+	private void declareNonDirect(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
+			final String[] attributes) {
+
+		if (streams != null) {
+			for (String stream : streams) {
+				declarer.declareStream(stream, false, new Fields(attributes));
+			}
+		} else {
+			declarer.declare(false, new Fields(attributes));
+		}
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testUndeclared() {
+		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+		declarer.getOutputType("unknownStreamId");
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareDirect() {
+		new FlinkOutputFieldsDeclarer().declare(true, null);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareDirect2() {
+		new FlinkOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, null);
+	}
+
+	@Test
+	public void testGetGroupingFieldIndexes() {
+		final int numberOfAttributes = 5 + this.r.nextInt(21);
+		final String[] attributes = new String[numberOfAttributes];
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			attributes[i] = "a" + i;
+		}
+
+		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+		declarer.declare(new Fields(attributes));
+
+		final int numberOfKeys = 1 + this.r.nextInt(25);
+		final LinkedList<String> groupingFields = new LinkedList<String>();
+		final boolean[] indexes = new boolean[numberOfAttributes];
+
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			if (this.r.nextInt(26) < numberOfKeys) {
+				groupingFields.add(attributes[i]);
+				indexes[i] = true;
+			} else {
+				indexes[i] = false;
+			}
+		}
+
+		final int[] expectedResult = new int[groupingFields.size()];
+		int j = 0;
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			if (indexes[i]) {
+				expectedResult[j++] = i;
+			}
+		}
+
+		final int[] result = declarer.getGroupingFieldIndexes(Utils.DEFAULT_STREAM_ID,
+				groupingFields);
+
+		Assert.assertEquals(expectedResult.length, result.length);
+		for (int i = 0; i < expectedResult.length; ++i) {
+			Assert.assertEquals(expectedResult[i], result[i]);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java
new file mode 100644
index 0000000..c3cb7d7
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelectorTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.Iterator;
+
+import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector;
+import org.apache.flink.stormcompatibility.util.SplitStreamType;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FlinkStormStreamSelectorTest {
+
+	@Test
+	public void testSelector() {
+		FlinkStormStreamSelector<Object> selector = new FlinkStormStreamSelector<Object>();
+		SplitStreamType<Object> tuple = new SplitStreamType<Object>();
+		Iterator<String> result;
+
+		tuple.streamId = "stream1";
+		result = selector.select(tuple).iterator();
+		Assert.assertEquals("stream1", result.next());
+		Assert.assertFalse(result.hasNext());
+
+		tuple.streamId = "stream2";
+		result = selector.select(tuple).iterator();
+		Assert.assertEquals("stream2", result.next());
+		Assert.assertFalse(result.hasNext());
+
+		tuple.streamId = "stream1";
+		result = selector.select(tuple).iterator();
+		Assert.assertEquals("stream1", result.next());
+		Assert.assertFalse(result.hasNext());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContextTest.java
----------------------------------------------------------------------
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 0000000..bd9ea3f
--- /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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), 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<String, SpoutSpec>(),
+				new HashMap<String, Bolt>(), new HashMap<String, StateSpoutSpec>()), null, null,
+				null, null, null, null, null, null, null, null, null, null, null, null, null)
+		.setSubscribedState(null, null, null);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummyBolt.java
----------------------------------------------------------------------
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 ec48719..b499373 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.IRichBolt;
 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() {}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestDummySpout.java
----------------------------------------------------------------------
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 62705b8..345ca12 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.task.TopologyContext;
 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 class TestDummySpout implements IRichSpout {
 	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) {}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/util/TestSink.java
----------------------------------------------------------------------
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 5699219..c8e5584 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 @@ import backtype.storm.tuple.Tuple;
 public class TestSink implements IRichBolt {
 	private static final long serialVersionUID = 4314871456719370877L;
 
+	public final static List<TopologyContext> result = new LinkedList<TopologyContext>();
+
 	@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() {}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapperTest.java
----------------------------------------------------------------------
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 381e130..b44e8a1 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 class FiniteStormSpoutWrapperTest {
 
 		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<Object>(stormSpout);
 		wrapper.setRuntimeContext(taskContext);
@@ -59,6 +63,8 @@ public class FiniteStormSpoutWrapperTest {
 
 		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<Object>(stormSpout);
 		wrapper.setRuntimeContext(taskContext);

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteTestSpout.java
----------------------------------------------------------------------
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/wrappers/FiniteTestSpout.java
deleted file mode 100644
index eef35cf..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FiniteTestSpout.java
+++ /dev/null
@@ -1,77 +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 backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.tuple.Values;
-
-import java.util.Map;
-
-class FiniteTestSpout implements IRichSpout {
-	private static final long serialVersionUID = 7992419478267824279L;
-
-	private int numberOfOutputTuples;
-	private SpoutOutputCollector collector;
-
-	public FiniteTestSpout(final int numberOfOutputTuples) {
-		this.numberOfOutputTuples = numberOfOutputTuples;
-	}
-
-	@SuppressWarnings("rawtypes")
-	@Override
-	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
-		this.collector = collector;
-	}
-
-	@Override
-	public void close() {/* nothing to do */}
-
-	@Override
-	public void activate() {/* nothing to do */}
-
-	@Override
-	public void deactivate() {/* nothing to do */}
-
-	@Override
-	public void nextTuple() {
-		if (--this.numberOfOutputTuples >= 0) {
-			this.collector.emit(new Values(new Integer(this.numberOfOutputTuples)));
-		}
-	}
-
-	@Override
-	public void ack(final Object msgId) {/* nothing to do */}
-
-	@Override
-	public void fail(final Object msgId) {/* nothing to do */}
-
-	@Override
-	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
-		declarer.declare(new Fields("dummy"));
-	}
-
-	@Override
-	public Map<String, Object> getComponentConfiguration() {
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java
----------------------------------------------------------------------
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/wrappers/FlinkStormStreamSelectorTest.java
deleted file mode 100644
index c0a6ed3..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/FlinkStormStreamSelectorTest.java
+++ /dev/null
@@ -1,51 +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.Iterator;
-
-import org.apache.flink.stormcompatibility.util.FlinkStormStreamSelector;
-import org.apache.flink.stormcompatibility.util.SplitStreamType;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class FlinkStormStreamSelectorTest {
-
-	@Test
-	public void testSelector() {
-		FlinkStormStreamSelector<Object> selector = new FlinkStormStreamSelector<Object>();
-		SplitStreamType<Object> tuple = new SplitStreamType<Object>();
-		Iterator<String> result;
-
-		tuple.streamId = "stream1";
-		result = selector.select(tuple).iterator();
-		Assert.assertEquals("stream1", result.next());
-		Assert.assertFalse(result.hasNext());
-
-		tuple.streamId = "stream2";
-		result = selector.select(tuple).iterator();
-		Assert.assertEquals("stream2", result.next());
-		Assert.assertFalse(result.hasNext());
-
-		tuple.streamId = "stream1";
-		result = selector.select(tuple).iterator();
-		Assert.assertEquals("stream1", result.next());
-		Assert.assertFalse(result.hasNext());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java
new file mode 100644
index 0000000..738eb1e
--- /dev/null
+++ b/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarerTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+import org.apache.flink.stormcompatibility.util.AbstractTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.ArrayList;
+
+public class SetupOutputFieldsDeclarerTest extends AbstractTest {
+
+	@Test
+	public void testDeclare() {
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+
+		int numberOfAttributes = this.r.nextInt(26);
+		declarer.declare(createSchema(numberOfAttributes));
+		Assert.assertEquals(1, declarer.outputSchemas.size());
+		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(Utils.DEFAULT_STREAM_ID)
+				.intValue());
+
+		final String sid = "streamId";
+		numberOfAttributes = this.r.nextInt(26);
+		declarer.declareStream(sid, createSchema(numberOfAttributes));
+		Assert.assertEquals(2, declarer.outputSchemas.size());
+		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(sid).intValue());
+	}
+
+	private Fields createSchema(final int numberOfAttributes) {
+		final ArrayList<String> schema = new ArrayList<String>(numberOfAttributes);
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			schema.add("a" + i);
+		}
+		return new Fields(schema);
+	}
+
+	@Test
+	public void testDeclareDirect() {
+		new SetupOutputFieldsDeclarer().declare(false, new Fields());
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareDirectFail() {
+		new SetupOutputFieldsDeclarer().declare(true, new Fields());
+	}
+
+	@Test
+	public void testDeclareStream() {
+		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, new Fields());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareStreamFail() {
+		new SetupOutputFieldsDeclarer().declareStream(null, new Fields());
+	}
+
+	@Test
+	public void testDeclareFullStream() {
+		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareFullStreamFailNonDefaultStream() {
+		new SetupOutputFieldsDeclarer().declareStream(null, false, new Fields());
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareFullStreamFailDirect() {
+		new SetupOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, new Fields());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapperTest.java
----------------------------------------------------------------------
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 5cfb151..6817593 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<Object, Object>(mock(IRichBolt.class),
 				new String[] { Utils.DEFAULT_STREAM_ID });
@@ -71,26 +71,26 @@ public class StormBoltWrapperTest extends AbstractTest {
 
 	@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<Object, Object>(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<Object, Object>(mock(IRichBolt.class), new String[] {});
 	}
@@ -133,12 +133,14 @@ public class StormBoltWrapperTest extends AbstractTest {
 
 		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 class StormBoltWrapperTest extends AbstractTest {
 
 		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 class StormBoltWrapperTest extends AbstractTest {
 
 		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<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
@@ -249,8 +256,11 @@ public class StormBoltWrapperTest extends AbstractTest {
 
 		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<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
 		wrapper.setup(mock(Output.class), taskContext);
 
@@ -275,9 +285,9 @@ public class StormBoltWrapperTest extends AbstractTest {
 	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<Object, Object> wrapper = new StormBoltWrapper<Object, Object>(bolt);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapperTest.java
----------------------------------------------------------------------
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 a4eea7e..77f1b05 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 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;
@@ -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 class StormFiniteSpoutWrapperTest extends AbstractTest {
 
 		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<Tuple1<Integer>> spoutWrapper = new StormFiniteSpoutWrapper<Tuple1<Integer>>(
@@ -94,11 +100,12 @@ public class StormFiniteSpoutWrapperTest extends AbstractTest {
 
 		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<Tuple1<Integer>> spoutWrapper = new StormFiniteSpoutWrapper<Tuple1<Integer>>(
 				spout);
-		spoutWrapper.setRuntimeContext(taskContext);
 
 		spoutWrapper.cancel();
 		final TestContext collector = new TestContext();

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
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/StormOutputFieldsDeclarerTest.java
deleted file mode 100644
index 561939f..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormOutputFieldsDeclarerTest.java
+++ /dev/null
@@ -1,91 +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.tuple.Fields;
-import backtype.storm.utils.Utils;
-
-import org.apache.flink.stormcompatibility.util.AbstractTest;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.ArrayList;
-
-public class StormOutputFieldsDeclarerTest extends AbstractTest {
-
-	@Test
-	public void testDeclare() {
-		final StormOutputFieldsDeclarer declarer = new StormOutputFieldsDeclarer();
-
-		int numberOfAttributes = this.r.nextInt(26);
-		declarer.declare(createSchema(numberOfAttributes));
-		Assert.assertEquals(1, declarer.outputSchemas.size());
-		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(Utils.DEFAULT_STREAM_ID)
-				.intValue());
-
-		final String sid = "streamId";
-		numberOfAttributes = this.r.nextInt(26);
-		declarer.declareStream(sid, createSchema(numberOfAttributes));
-		Assert.assertEquals(2, declarer.outputSchemas.size());
-		Assert.assertEquals(numberOfAttributes, declarer.outputSchemas.get(sid).intValue());
-	}
-
-	private Fields createSchema(final int numberOfAttributes) {
-		final ArrayList<String> schema = new ArrayList<String>(numberOfAttributes);
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			schema.add("a" + i);
-		}
-		return new Fields(schema);
-	}
-
-	@Test
-	public void testDeclareDirect() {
-		new StormOutputFieldsDeclarer().declare(false, new Fields());
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareDirectFail() {
-		new StormOutputFieldsDeclarer().declare(true, new Fields());
-	}
-
-	@Test
-	public void testDeclareStream() {
-		new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, new Fields());
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareStreamFail() {
-		new StormOutputFieldsDeclarer().declareStream(null, new Fields());
-	}
-
-	@Test
-	public void testDeclareFullStream() {
-		new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, false, new Fields());
-	}
-
-	@Test(expected = IllegalArgumentException.class)
-	public void testDeclareFullStreamFailNonDefaultStream() {
-		new StormOutputFieldsDeclarer().declareStream(null, false, new Fields());
-	}
-
-	@Test(expected = UnsupportedOperationException.class)
-	public void testDeclareFullStreamFailDirect() {
-		new StormOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, new Fields());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutWrapperTest.java
----------------------------------------------------------------------
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 04dc48d..f4fb4da 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.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;
@@ -89,8 +91,12 @@ public class StormSpoutWrapperTest extends AbstractTest {
 
 		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<Tuple1<Integer>> spoutWrapper = new StormSpoutWrapper<Tuple1<Integer>>(spout);
 		spoutWrapper.setRuntimeContext(taskContext);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/test/java/org/apache/flink/stormcompatibility/wrappers/StormWrapperSetupHelperTest.java
----------------------------------------------------------------------
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 7497ffc..c799d63 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 class StormWrapperSetupHelperTest extends AbstractTest {
 			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 class StormWrapperSetupHelperTest extends AbstractTest {
 			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 @@ public class StormWrapperSetupHelperTest extends AbstractTest {
 			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<String, Integer> attributes = new HashMap<String, Integer>();
 		attributes.put(Utils.DEFAULT_STREAM_ID, numberOfAttributes);
@@ -132,4 +149,167 @@ public class StormWrapperSetupHelperTest extends AbstractTest {
 						.newHashSet(new String[] { Utils.DEFAULT_STREAM_ID }) : null));
 	}
 
+	@Test
+	public void testCreateTopologyContext() {
+		HashMap<String, Integer> dops = new HashMap<String, Integer>();
+		dops.put("spout1", 1);
+		dops.put("spout2", 3);
+		dops.put("bolt1", 1);
+		dops.put("bolt2", 2);
+		dops.put("sink", 1);
+
+		HashMap<String, Integer> taskCounter = new HashMap<String, Integer>();
+		taskCounter.put("spout1", 0);
+		taskCounter.put("spout2", 0);
+		taskCounter.put("bolt1", 0);
+		taskCounter.put("bolt2", 0);
+		taskCounter.put("sink", 0);
+
+		HashMap<String, IComponent> operators = new HashMap<String, IComponent>();
+		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<Integer> taskIds = new HashSet<Integer>();
+
+		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<Integer, String> taskToComponents = new HashMap<Integer, String>();
+			Set<Integer> allTaskIds = new HashSet<Integer>();
+			for (String componentId : expectedContext.getComponentIds()) {
+				List<Integer> possibleTasks = expectedContext.getComponentTasks(componentId);
+				List<Integer> tasks = topologyContext.getComponentTasks(componentId);
+
+				Iterator<Integer> p_it = possibleTasks.iterator();
+				Iterator<Integer> 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 */ }
+		}
+	}
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormFileSpout.java
----------------------------------------------------------------------
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 64b3e28..d3776fb 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.spout.SpoutOutputCollector;
 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;

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormInMemorySpout.java
----------------------------------------------------------------------
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 6fb764d..5efff66 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

http://git-wip-us.apache.org/repos/asf/flink/blob/7a67a60f/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-examples/src/test/java/org/apache/flink/stormcompatibility/split/SplitBoltTopology.java
----------------------------------------------------------------------
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 c992b6b..5f637d3 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 class SplitBoltTopology {
 			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;


[06/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java
new file mode 100644
index 0000000..985cd68
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationLocal.java
@@ -0,0 +1,75 @@
+/*
+ * 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.excamation;
+
+import backtype.storm.Config;
+import backtype.storm.utils.Utils;
+
+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}.
+ * <p/>
+ * 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).
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>ExclamationLocal &lt;text path&gt; &lt;result path&gt;</code><br/>
+ * If no parameters are provided, the program is run with default data from
+ * {@link org.apache.flink.examples.java.wordcount.util.WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>run a regular Storm program locally on Flink</li>
+ * </ul>
+ */
+public class ExclamationLocal {
+
+	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 locally
+		Config conf = new Config();
+		conf.put(ExclamationBolt.EXCLAMATION_COUNT, ExclamationTopology.getExclamation());
+		final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster();
+		cluster.submitTopology(topologyId, conf, builder.createTopology());
+
+		Utils.sleep(10 * 1000);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java
new file mode 100644
index 0000000..70d25a2
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationTopology.java
@@ -0,0 +1,123 @@
+/*
+ * 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.excamation;
+
+import org.apache.flink.examples.java.wordcount.util.WordCountData;
+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 two exclamation marks to every line of a text files in a streaming
+ * fashion. The program is constructed as a regular {@link StormTopology}.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>Exclamation[Local|RemoteByClient|RemoteBySubmitter] &lt;text path&gt;
+ * &lt;result path&gt;</code><br/>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>construct a regular Storm topology as Flink program</li>
+ * <li>make use of the FiniteSpout interface</li>
+ * </ul>
+ */
+public class ExclamationTopology {
+
+	public final static String spoutId = "source";
+	public final static String firstBoltId = "exclamation1";
+	public final static String secondBoltId = "exclamation2";
+	public final static String sinkId = "sink";
+	private final static OutputFormatter formatter = new SimpleOutputFormatter();
+
+	public static FlinkTopologyBuilder buildTopology() {
+		final FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
+
+		// get input data
+		if (fileInputOutput) {
+			// read the text file from given input path
+			final String[] tokens = textPath.split(":");
+			final String inputFile = tokens[tokens.length - 1];
+			builder.setSpout(spoutId, new FiniteFileSpout(inputFile));
+		} else {
+			builder.setSpout(spoutId, new FiniteInMemorySpout(WordCountData.WORDS));
+		}
+
+		builder.setBolt(firstBoltId, new ExclamationBolt(), 3).shuffleGrouping(spoutId);
+		builder.setBolt(secondBoltId, new ExclamationBolt(), 2).shuffleGrouping(firstBoltId);
+
+		// 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 BoltFileSink(outputFile, formatter))
+			.shuffleGrouping(secondBoltId);
+		} else {
+			builder.setBolt(sinkId, new BoltPrintSink(formatter), 4)
+			.shuffleGrouping(secondBoltId);
+		}
+
+		return builder;
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileInputOutput = false;
+	private static String textPath;
+	private static String outputPath;
+	private static int exclamationNum = 3;
+
+	static int getExclamation() {
+		return exclamationNum;
+	}
+
+	static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileInputOutput = true;
+			if (args.length == 3) {
+				textPath = args[0];
+				outputPath = args[1];
+				exclamationNum = Integer.parseInt(args[2]);
+			} else {
+				System.err.println("Usage: StormExclamation* <text path> <result path>  <number of exclamation marks>");
+				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 <text path> <result path> <number of exclamation marks>");
+		}
+
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java
new file mode 100644
index 0000000..01ab907
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithBolt.java
@@ -0,0 +1,144 @@
+/*
+ * 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.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.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 3+x exclamation marks to every line of a text files in a streaming
+ * fashion. The program is constructed as a regular {@link StormTopology}.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage:
+ * <code>ExclamationWithmBolt &lt;text path&gt; &lt;result path&gt; &lt;number of exclamation marks&gt;</code><br/>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData} with x=2.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>use a Bolt within a Flink Streaming program</li>
+ * <li>how to configure a Bolt using StormConfig</li>
+ * </ul>
+ */
+public class ExclamationWithBolt {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// set Storm configuration
+		StormConfig config = new StormConfig();
+		config.put(ExclamationBolt.EXCLAMATION_COUNT, new Integer(exclamationNum));
+		env.getConfig().setGlobalJobParameters(config);
+
+		// get input data
+		final DataStream<String> text = getTextDataStream(env);
+
+		final DataStream<String> exclaimed = text
+				.transform("StormBoltTokenizer",
+						TypeExtractor.getForObject(""),
+						new BoltWrapper<String, String>(new ExclamationBolt(),
+								new String[] { Utils.DEFAULT_STREAM_ID }))
+								.map(new ExclamationMap());
+
+		// emit result
+		if (fileOutput) {
+			exclaimed.writeAsText(outputPath);
+		} else {
+			exclaimed.print();
+		}
+
+		// execute program
+		env.execute("Streaming WordCount with bolt tokenizer");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	private static class ExclamationMap implements MapFunction<String, String> {
+		private static final long serialVersionUID = 4614754344067170619L;
+
+		@Override
+		public String map(String value) throws Exception {
+			return value + "!!!";
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+	private static int exclamationNum = 2;
+
+	private static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 3) {
+				textPath = args[0];
+				outputPath = args[1];
+				exclamationNum = Integer.parseInt(args[2]);
+			} else {
+				System.err.println("Usage: ExclamationWithBolt <text path> <result path> <number of exclamation marks>");
+				return false;
+			}
+		} else {
+			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: ExclamationWithBolt <text path> <result path> <number of exclamation marks>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		}
+
+		return env.fromElements(WordCountData.WORDS);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java
new file mode 100644
index 0000000..22938e5
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/ExclamationWithSpout.java
@@ -0,0 +1,150 @@
+/*
+ * 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.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.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 six exclamation marks to every line of a text files in a streaming
+ * fashion. The program is constructed as a regular {@link StormTopology}.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>ExclamationWithSpout &lt;text path&gt; &lt;result path&gt;</code><br/>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>use a Storm spout within a Flink Streaming program</li>
+ * <li>make use of the FiniteSpout interface</li>
+ * <li>make use of the FiniteSpout interface</li>
+ * <li>how to configure a Spout using StormConfig</li>
+ * </ul>
+ */
+public class ExclamationWithSpout {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		final DataStream<String> text = getTextDataStream(env);
+
+		final DataStream<String> exclaimed = text
+				.map(new ExclamationMap())
+				.map(new ExclamationMap());
+
+		// emit result
+		if (fileOutput) {
+			exclaimed.writeAsText(outputPath);
+		} else {
+			exclaimed.print();
+		}
+
+		// execute program
+		env.execute("Streaming Exclamation with Storm spout source");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	private static class ExclamationMap implements MapFunction<String, String> {
+		private static final long serialVersionUID = -684993133807698042L;
+
+		@Override
+		public String map(String value) throws Exception {
+			return value + "!!!";
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: ExclamationWithSpout <text path> <result path>");
+				return false;
+			}
+		} else {
+			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: ExclamationWithSpout <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			final String[] tokens = textPath.split(":");
+			final String inputFile = tokens[tokens.length - 1];
+
+			// set Storm configuration
+			StormConfig config = new StormConfig();
+			config.put(FiniteFileSpout.INPUT_FILE_PATH, inputFile);
+			env.getConfig().setGlobalJobParameters(config);
+
+			return env.addSource(
+					new SpoutWrapper<String>(new FiniteFileSpout(),
+							new String[] { Utils.DEFAULT_STREAM_ID }),
+							TypeExtractor.getForClass(String.class)).setParallelism(1);
+		}
+
+		return env.addSource(
+				new SpoutWrapper<String>(new FiniteInMemorySpout(
+						WordCountData.WORDS), new String[] { Utils.DEFAULT_STREAM_ID }),
+						TypeExtractor.getForClass(String.class)).setParallelism(1);
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators/ExclamationBolt.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators/ExclamationBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators/ExclamationBolt.java
new file mode 100644
index 0000000..cfc49a1
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/excamation/operators/ExclamationBolt.java
@@ -0,0 +1,75 @@
+/*
+ * 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.excamation.operators;
+
+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.tuple.Values;
+
+import java.util.Map;
+
+public class ExclamationBolt implements IRichBolt {
+	private final static long serialVersionUID = -6364882114201311380L;
+
+	public final static String EXCLAMATION_COUNT = "exclamation.count";
+
+	private OutputCollector collector;
+	private String exclamation;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(Map conf, TopologyContext context, OutputCollector collector) {
+		this.collector = collector;
+
+		Object count = conf.get(EXCLAMATION_COUNT);
+		if (count != null) {
+			int exclamationNum = (Integer) count;
+			StringBuilder builder = new StringBuilder();
+			for (int index = 0; index < exclamationNum; ++index) {
+				builder.append('!');
+			}
+			this.exclamation = builder.toString();
+		} else {
+			this.exclamation = "!";
+		}
+	}
+
+	@Override
+	public void cleanup() {
+	}
+
+	@Override
+	public void execute(Tuple tuple) {
+		collector.emit(tuple, new Values(tuple.getString(0) + this.exclamation));
+	}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields("word"));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
new file mode 100644
index 0000000..560fe51
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/SpoutSplitExample.java
@@ -0,0 +1,102 @@
+/*
+ * 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.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.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.
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>handle multiple output stream of a spout</li>
+ * <li>accessing each stream by .split(...) and .select(...)</li>
+ * <li>strip wrapper data type SplitStreamType for further processing in Flink</li>
+ * </ul>
+ * <p/>
+ * This example would work the same way for multiple bolt output streams.
+ */
+public class SpoutSplitExample {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		String[] rawOutputs = new String[] { RandomSpout.EVEN_STREAM, RandomSpout.ODD_STREAM };
+
+		final DataStream<SplitStreamType<Integer>> numbers = env.addSource(
+				new SpoutWrapper<SplitStreamType<Integer>>(new RandomSpout(true, 0),
+						rawOutputs), TypeExtractor.getForObject(new SplitStreamType<Integer>()));
+
+		SplitStream<SplitStreamType<Integer>> splitStream = numbers
+				.split(new StormStreamSelector<Integer>());
+
+		DataStream<SplitStreamType<Integer>> evenStream = splitStream.select(RandomSpout.EVEN_STREAM);
+		DataStream<SplitStreamType<Integer>> oddStream = splitStream.select(RandomSpout.ODD_STREAM);
+
+		evenStream.map(new SplitStreamMapper<Integer>()).returns(Integer.class).map(new Enrich("even")).print();
+		oddStream.transform("oddBolt",
+				TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
+				new BoltWrapper<SplitStreamType<Integer>, Tuple2<String, Integer>>(
+						new VerifyAndEnrichBolt(false)))
+						.print();
+
+		// execute program
+		env.execute("Spout split stream example");
+	}
+
+	// *************************************************************************
+	//     USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Same as {@link VerifyAndEnrichBolt}.
+	 */
+	private final static class Enrich implements MapFunction<Integer, Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 5213888269197438892L;
+		private final Tuple2<String, Integer> out;
+
+		public Enrich(String token) {
+			this.out = new Tuple2<String, Integer>(token, 0);
+		}
+
+		@Override
+		public Tuple2<String, Integer> map(Integer value) throws Exception {
+			this.out.setField(value, 1);
+			return this.out;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java
new file mode 100644
index 0000000..d315395
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java
@@ -0,0 +1,76 @@
+/*
+ * 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.split.operators;
+
+import java.util.Map;
+import java.util.Random;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichSpout;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Values;
+
+public class RandomSpout extends BaseRichSpout {
+	private static final long serialVersionUID = -3978554318742509334L;
+
+	public static final String EVEN_STREAM = "even";
+	public static final String ODD_STREAM = "odd";
+
+	private final boolean split;
+	private Random r = new Random();
+	private SpoutOutputCollector collector;
+
+	public RandomSpout(boolean split, long seed) {
+		this.split = split;
+		this.r = new Random(seed);
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void nextTuple() {
+		int i = r.nextInt();
+		if (split) {
+			if (i % 2 == 0) {
+				this.collector.emit(EVEN_STREAM, new Values(i));
+			} else {
+				this.collector.emit(ODD_STREAM, new Values(i));
+			}
+		} else {
+			this.collector.emit(new Values(i));
+		}
+	}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {
+		Fields schema = new Fields("number");
+		if (split) {
+			declarer.declareStream(EVEN_STREAM, schema);
+			declarer.declareStream(ODD_STREAM, schema);
+		} else {
+			declarer.declare(schema);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java
new file mode 100644
index 0000000..99fec4d
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java
@@ -0,0 +1,61 @@
+/*
+ * 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.split.operators;
+
+import java.util.Map;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.topology.base.BaseRichBolt;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.Tuple;
+import backtype.storm.tuple.Values;
+
+public class VerifyAndEnrichBolt extends BaseRichBolt {
+	private static final long serialVersionUID = -7277395570966328721L;
+
+	private final boolean evenOrOdd; // true: even -- false: odd
+	private final String token;
+	private OutputCollector collector;
+
+	public VerifyAndEnrichBolt(boolean evenOrOdd) {
+		this.evenOrOdd = evenOrOdd;
+		this.token = evenOrOdd ? "even" : "odd";
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void execute(Tuple input) {
+		if ((input.getInteger(0) % 2 == 0) != this.evenOrOdd) {
+			throw new RuntimeException("Invalid number detected.");
+		}
+		this.collector.emit(new Values(this.token, input.getInteger(0)));
+	}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields("evenOrOdd", "number"));
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java
new file mode 100644
index 0000000..a6c61d4
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java
@@ -0,0 +1,76 @@
+/*
+ * 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.util;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+
+import java.util.Map;
+
+/**
+ * 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 AbstractBoltSink implements IRichBolt {
+	private static final long serialVersionUID = -1626323806848080430L;
+
+	private StringBuilder lineBuilder;
+	private String prefix = "";
+	private final OutputFormatter formatter;
+
+	public AbstractBoltSink(final OutputFormatter formatter) {
+		this.formatter = formatter;
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public final void prepare(final Map stormConf, final TopologyContext context,
+			final OutputCollector collector) {
+		this.prepareSimple(stormConf, context);
+		if (context.getComponentCommon(context.getThisComponentId()).get_parallelism_hint() > 1) {
+			this.prefix = context.getThisTaskId() + "> ";
+		}
+	}
+
+	protected abstract void prepareSimple(final Map<?, ?> stormConf, final TopologyContext context);
+
+	@Override
+	public final void execute(final Tuple input) {
+		this.lineBuilder = new StringBuilder();
+		this.lineBuilder.append(this.prefix);
+		this.lineBuilder.append(this.formatter.format(input));
+		this.writeExternal(this.lineBuilder.toString());
+	}
+
+	protected abstract void writeExternal(final String line);
+
+	@Override
+	public void cleanup() {/* nothing to do */}
+
+	@Override
+	public final void declareOutputFields(final OutputFieldsDeclarer declarer) {/* nothing to do */}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java
new file mode 100644
index 0000000..d19ffbf
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java
@@ -0,0 +1,70 @@
+/*
+ * 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.util;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+
+import java.util.Map;
+
+/**
+ * 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 AbstractLineSpout implements IRichSpout {
+	private static final long serialVersionUID = 8876828403487806771L;
+
+	public final static String ATTRIBUTE_LINE = "line";
+
+	protected SpoutOutputCollector collector;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
+		this.collector = collector;
+	}
+
+	@Override
+	public void close() {/* noting to do */}
+
+	@Override
+	public void activate() {/* noting to do */}
+
+	@Override
+	public void deactivate() {/* noting to do */}
+
+	@Override
+	public void ack(final Object msgId) {/* noting to do */}
+
+	@Override
+	public void fail(final Object msgId) {/* noting to do */}
+
+	@Override
+	public void declareOutputFields(final OutputFieldsDeclarer declarer) {
+		declarer.declare(new Fields(ATTRIBUTE_LINE));
+	}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java
new file mode 100644
index 0000000..5cd3f68
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java
@@ -0,0 +1,76 @@
+/*
+ * 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.util;
+
+import backtype.storm.task.TopologyContext;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * 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 BoltFileSink extends AbstractBoltSink {
+	private static final long serialVersionUID = 2014027288631273666L;
+
+	private final String path;
+	private BufferedWriter writer;
+
+	public BoltFileSink(final String path) {
+		this(path, new SimpleOutputFormatter());
+	}
+
+	public BoltFileSink(final String path, final OutputFormatter formatter) {
+		super(formatter);
+		this.path = path;
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepareSimple(final Map stormConf, final TopologyContext context) {
+		try {
+			this.writer = new BufferedWriter(new FileWriter(this.path));
+		} catch (final IOException e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	@Override
+	public void writeExternal(final String line) {
+		try {
+			this.writer.write(line + "\n");
+		} catch (final IOException e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	@Override
+	public void cleanup() {
+		if (this.writer != null) {
+			try {
+				this.writer.close();
+			} catch (final IOException e) {
+				throw new RuntimeException(e);
+			}
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java
new file mode 100644
index 0000000..044246b
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java
@@ -0,0 +1,45 @@
+/*
+ * 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.util;
+
+import backtype.storm.task.TopologyContext;
+
+import java.util.Map;
+
+/**
+ * Implements a sink that prints the received data to {@code stdout}.
+ */
+public final class BoltPrintSink extends AbstractBoltSink {
+	private static final long serialVersionUID = -6650011223001009519L;
+
+	public BoltPrintSink(OutputFormatter formatter) {
+		super(formatter);
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepareSimple(final Map stormConf, final TopologyContext context) {
+		/* nothing to do */
+	}
+
+	@Override
+	public void writeExternal(final String line) {
+		System.out.println(line);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java
new file mode 100644
index 0000000..1126a2a
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java
@@ -0,0 +1,88 @@
+/*
+ * 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.util;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Values;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Implements a Spout that reads data from a given local file.
+ */
+public class FileSpout extends AbstractLineSpout {
+	private static final long serialVersionUID = -6996907090003590436L;
+
+	public final static String INPUT_FILE_PATH = "input.path";
+
+	protected String path = null;
+	protected BufferedReader reader;
+
+	public FileSpout() {}
+
+	public FileSpout(final String path) {
+		this.path = path;
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
+		super.open(conf, context, collector);
+
+		Object configuredPath = conf.get(INPUT_FILE_PATH);
+		if(configuredPath != null) {
+			this.path = (String)configuredPath;
+		}
+
+		try {
+			this.reader = new BufferedReader(new FileReader(this.path));
+		} catch (final FileNotFoundException e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	@Override
+	public void close() {
+		if (this.reader != null) {
+			try {
+				this.reader.close();
+			} catch (final IOException e) {
+				throw new RuntimeException(e);
+			}
+		}
+	}
+
+	@Override
+	public void nextTuple() {
+		String line;
+		try {
+			line = this.reader.readLine();
+			if (line != null) {
+				this.collector.emit(new Values(line));
+			}
+		} catch (final IOException e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java
new file mode 100644
index 0000000..75450c4
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java
@@ -0,0 +1,77 @@
+/*
+ * 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.util;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.tuple.Values;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.flink.storm.util.FiniteSpout;
+
+/**
+ * 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 FiniteFileSpout extends FileSpout implements FiniteSpout {
+	private static final long serialVersionUID = -1472978008607215864L;
+
+	private String line;
+	private boolean newLineRead;
+
+	public FiniteFileSpout() {}
+
+	public FiniteFileSpout(String path) {
+		super(path);
+	}
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void open(final Map conf, final TopologyContext context, final SpoutOutputCollector collector) {
+		super.open(conf, context, collector);
+		newLineRead = false;
+	}
+
+	@Override
+	public void nextTuple() {
+		this.collector.emit(new Values(line));
+		newLineRead = false;
+	}
+
+	/**
+	 * Can be called before nextTuple() any times including 0.
+	 */
+	@Override
+	public boolean reachedEnd() {
+		try {
+			readLine();
+		} catch (IOException e) {
+			throw new RuntimeException("Exception occured while reading file " + path);
+		}
+		return line == null;
+	}
+
+	private void readLine() throws IOException {
+		if (!newLineRead) {
+			line = reader.readLine();
+			newLineRead = true;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java
new file mode 100644
index 0000000..1490872
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteInMemorySpout.java
@@ -0,0 +1,40 @@
+/*
+ * 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.util;
+
+import org.apache.flink.storm.util.FiniteSpout;
+
+
+/**
+ * Implements a Spout that reads String[] data stored in memory. The Spout stops automatically when it emitted all of
+ * the data.
+ */
+public class FiniteInMemorySpout extends InMemorySpout<String> implements FiniteSpout {
+	private static final long serialVersionUID = -4008858647468647019L;
+
+	public FiniteInMemorySpout(String[] source) {
+		super(source);
+	}
+
+	@Override
+	public boolean reachedEnd() {
+		return counter >= source.length;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java
new file mode 100644
index 0000000..5e4c7ba
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java
@@ -0,0 +1,42 @@
+/*
+ * 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.util;
+
+import backtype.storm.tuple.Values;
+
+/**
+ * Implements a Spout that reads data stored in memory.
+ */
+public class InMemorySpout<T> extends AbstractLineSpout {
+	private static final long serialVersionUID = -4008858647468647019L;
+
+	protected T[] source;
+	protected int counter = 0;
+
+	public InMemorySpout(T[] source) {
+		this.source = source;
+	}
+
+	@Override
+	public void nextTuple() {
+		if (this.counter < source.length) {
+			this.collector.emit(new Values(source[this.counter++]));
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java
new file mode 100644
index 0000000..e696f9b
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java
@@ -0,0 +1,37 @@
+/*
+ * 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.util;
+
+import backtype.storm.tuple.Tuple;
+
+import java.io.Serializable;
+
+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 console.
+	 * 
+	 * @param input
+	 *            The tuple to be formatted
+	 * @return The string result of the formatting
+	 */
+	public String format(Tuple input);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java
new file mode 100644
index 0000000..cef0081
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java
@@ -0,0 +1,41 @@
+/*
+ * 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.util;
+
+import backtype.storm.tuple.Tuple;
+
+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 console.
+	 * 
+	 * @param input
+	 *            The tuple to be formatted
+	 * @return The string result of the formatting
+	 */
+	@Override
+	public String format(final Tuple input) {
+		if (input.getValues().size() != 1) {
+			throw new RuntimeException("The output is not raw");
+		}
+		return input.getValue(0).toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java
new file mode 100644
index 0000000..5d7ba53
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java
@@ -0,0 +1,38 @@
+/*
+ * 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.util;
+
+import backtype.storm.tuple.Tuple;
+
+public class TupleOutputFormatter implements OutputFormatter {
+	private static final long serialVersionUID = -599665757723851761L;
+
+	@Override
+	public String format(final Tuple input) {
+		final StringBuilder stringBuilder = new StringBuilder();
+		stringBuilder.append("(");
+		for (final Object attribute : input.getValues()) {
+			stringBuilder.append(attribute);
+			stringBuilder.append(",");
+		}
+		stringBuilder.replace(stringBuilder.length() - 1, stringBuilder.length(), ")");
+		return stringBuilder.toString();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java
new file mode 100644
index 0000000..aa3a075
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.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.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 {@link IRichBolt Bolt}.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>use a Bolt within a Flink Streaming program.</li>
+ * </ul>
+ */
+public class BoltTokenizerWordCount {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		final DataStream<String> text = getTextDataStream(env);
+
+		final DataStream<Tuple2<String, Integer>> counts = text
+				// split up the lines in pairs (2-tuples) containing: (word,1)
+				// this is done by a bolt that is wrapped accordingly
+				.transform("BoltTokenizer",
+						TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
+						new BoltWrapper<String, Tuple2<String, Integer>>(new BoltTokenizer()))
+				// group by the tuple field "0" and sum up tuple field "1"
+				.keyBy(0).sum(1);
+
+		// emit result
+		if (fileOutput) {
+			counts.writeAsText(outputPath);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("Streaming WordCount with bolt tokenizer");
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: BoltTokenizerWordCount <text path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing BoltTokenizerWordCount example with built-in default data");
+			System.out.println("  Provide parameters to read input data from a file");
+			System.out.println("  Usage: BoltTokenizerWordCount <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			return env.readTextFile(textPath);
+		}
+
+		return env.fromElements(WordCountData.WORDS);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java
new file mode 100644
index 0000000..f72acb3
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java
@@ -0,0 +1,134 @@
+/*
+ * 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.wordcount;
+
+import backtype.storm.topology.IRichBolt;
+
+import org.apache.flink.api.java.io.CsvInputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.PojoTypeInfo;
+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.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 {@link IRichBolt Bolt}. In contrast to {@link BoltTokenizerWordCount}
+ * the tokenizer's input is a POJO type and the single field is accessed by name.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>how to access attributes by name within a Bolt for POJO type input streams
+ * </ul>
+ */
+public class BoltTokenizerWordCountPojo {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		final DataStream<Sentence> text = getTextDataStream(env);
+
+		final DataStream<Tuple2<String, Integer>> counts = text
+				// split up the lines in pairs (2-tuples) containing: (word,1)
+				// this is done by a bolt that is wrapped accordingly
+				.transform("BoltTokenizerPojo",
+						TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
+						new BoltWrapper<Sentence, Tuple2<String, Integer>>(new BoltTokenizerByName()))
+				// group by the tuple field "0" and sum up tuple field "1"
+				.keyBy(0).sum(1);
+
+		// emit result
+		if (fileOutput) {
+			counts.writeAsText(outputPath);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("Streaming WordCount with POJO bolt tokenizer");
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: BoltTokenizerWordCountPojo <text path> <result path>");
+				return false;
+			}
+		} else {
+			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: BoltTokenizerWordCountPojo <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<Sentence> getTextDataStream(final StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			PojoTypeInfo<Sentence> sourceType = (PojoTypeInfo<Sentence>) TypeExtractor
+					.getForObject(new Sentence(""));
+			return env.createInput(new CsvInputFormat<Sentence>(new Path(
+					textPath), CsvInputFormat.DEFAULT_LINE_DELIMITER,
+					CsvInputFormat.DEFAULT_LINE_DELIMITER, sourceType),
+					sourceType);
+		}
+
+		return env.fromElements(WordCountDataPojos.SENTENCES);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java
new file mode 100644
index 0000000..7617e95
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java
@@ -0,0 +1,137 @@
+/*
+ * 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.wordcount;
+
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.tuple.Fields;
+
+import org.apache.flink.api.java.io.CsvInputFormat;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+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.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 {@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.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>how to access attributes by name within a Bolt for {@link Tuple} type input streams
+ * </ul>
+ */
+public class BoltTokenizerWordCountWithNames {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		final DataStream<Tuple1<String>> text = getTextDataStream(env);
+
+		final DataStream<Tuple2<String, Integer>> 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(
+						"BoltTokenizerWithNames",
+						TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)),
+						new BoltWrapper<Tuple1<String>, Tuple2<String, Integer>>(
+								new BoltTokenizerByName(), new Fields("sentence")))
+				// group by the tuple field "0" and sum up tuple field "1"
+				.keyBy(0).sum(1);
+
+		// emit result
+		if (fileOutput) {
+			counts.writeAsText(outputPath);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("Streaming WordCount with schema bolt tokenizer");
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: BoltTokenizerWordCountWithNames <text path> <result path>");
+				return false;
+			}
+		} else {
+			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: BoltTokenizerWordCountWithNames <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<Tuple1<String>> getTextDataStream(final StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			TupleTypeInfo<Tuple1<String>> sourceType = (TupleTypeInfo<Tuple1<String>>)TypeExtractor
+					.getForObject(new Tuple1<String>(""));
+			return env.createInput(new CsvInputFormat<Tuple1<String>>(new Path(
+					textPath), CsvInputFormat.DEFAULT_LINE_DELIMITER,
+					CsvInputFormat.DEFAULT_LINE_DELIMITER, sourceType),
+					sourceType);
+		}
+
+		return env.fromElements(WordCountDataTuple.TUPLES);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java
new file mode 100644
index 0000000..bb451fe
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java
@@ -0,0 +1,157 @@
+/*
+ * 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.wordcount;
+
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.utils.Utils;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+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.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 {@link IRichSpout Spout}.
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCount &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>use a Spout within a Flink Streaming program.</li>
+ * </ul>
+ */
+public class SpoutSourceWordCount {
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!parseParameters(args)) {
+			return;
+		}
+
+		// set up the execution environment
+		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+		// get input data
+		final DataStream<String> text = getTextDataStream(env);
+
+		final DataStream<Tuple2<String, Integer>> counts =
+				// split up the lines in pairs (2-tuples) containing: (word,1)
+				text.flatMap(new Tokenizer())
+				// group by the tuple field "0" and sum up tuple field "1"
+				.keyBy(0).sum(1);
+
+		// emit result
+		if (fileOutput) {
+			counts.writeAsText(outputPath);
+		} else {
+			counts.print();
+		}
+
+		// execute program
+		env.execute("Streaming WordCount with spout source");
+	}
+
+	// *************************************************************************
+	// USER FUNCTIONS
+	// *************************************************************************
+
+	/**
+	 * Implements the string tokenizer that splits sentences into words as a user-defined FlatMapFunction. The function
+	 * takes a line (String) and splits it into multiple pairs in the form of "(word,1)" (Tuple2<String, Integer>).
+	 */
+	public static final class Tokenizer implements FlatMapFunction<String, Tuple2<String, Integer>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void flatMap(final String value, final Collector<Tuple2<String, Integer>> out) throws Exception {
+			// normalize and split the line
+			final String[] tokens = value.toLowerCase().split("\\W+");
+
+			// emit the pairs
+			for (final String token : tokens) {
+				if (token.length() > 0) {
+					out.collect(new Tuple2<String, Integer>(token, 1));
+				}
+			}
+		}
+	}
+
+	// *************************************************************************
+	// UTIL METHODS
+	// *************************************************************************
+
+	private static boolean fileOutput = false;
+	private static String textPath;
+	private static String outputPath;
+
+	private static boolean parseParameters(final String[] args) {
+
+		if (args.length > 0) {
+			// parse input arguments
+			fileOutput = true;
+			if (args.length == 2) {
+				textPath = args[0];
+				outputPath = args[1];
+			} else {
+				System.err.println("Usage: SpoutSourceWordCount <text path> <result path>");
+				return false;
+			}
+		} else {
+			System.out.println("Executing SpoutSourceWordCount example with built-in default data");
+			System.out.println("  Provide parameters to read input data from a file");
+			System.out.println("  Usage: SpoutSourceWordCount <text path> <result path>");
+		}
+		return true;
+	}
+
+	private static DataStream<String> getTextDataStream(final StreamExecutionEnvironment env) {
+		if (fileOutput) {
+			// read the text file from given input path
+			final String[] tokens = textPath.split(":");
+			final String localFile = tokens[tokens.length - 1];
+			return env.addSource(
+					new SpoutWrapper<String>(new WordCountFileSpout(localFile),
+							new String[] { Utils.DEFAULT_STREAM_ID }, -1),
+							TypeExtractor.getForClass(String.class)).setParallelism(1);
+		}
+
+		return env.addSource(
+				new SpoutWrapper<String>(new WordCountInMemorySpout(),
+						new String[] { Utils.DEFAULT_STREAM_ID }, -1),
+						TypeExtractor.getForClass(String.class)).setParallelism(1);
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java
new file mode 100644
index 0000000..18f49c1
--- /dev/null
+++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java
@@ -0,0 +1,76 @@
+/*
+ * 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.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.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}.
+ * <p/>
+ * This example shows how to run program directly within Java, thus it cannot be used to submit a {@link StormTopology}
+ * via Flink command line clients (ie, bin/flink).
+ * <p/>
+ * <p/>
+ * The input is a plain text file with lines separated by newline characters.
+ * <p/>
+ * <p/>
+ * Usage: <code>WordCountLocal &lt;text path&gt; &lt;result path&gt;</code><br>
+ * If no parameters are provided, the program is run with default data from {@link WordCountData}.
+ * <p/>
+ * <p/>
+ * This example shows how to:
+ * <ul>
+ * <li>run a regular Storm program locally on Flink</li>
+ * </ul>
+ */
+public class WordCountLocal {
+	public final static String topologyId = "Storm WordCount";
+
+	// *************************************************************************
+	// PROGRAM
+	// *************************************************************************
+
+	public static void main(final String[] args) throws Exception {
+
+		if (!WordCountTopology.parseParameters(args)) {
+			return;
+		}
+
+		// build Topology the Storm way
+		final FlinkTopologyBuilder builder = WordCountTopology.buildTopology();
+
+		// execute program locally
+		final FlinkLocalCluster cluster = FlinkLocalCluster.getLocalCluster();
+		cluster.submitTopology(topologyId, null, builder.createTopology());
+
+		Utils.sleep(10 * 1000);
+
+		// TODO kill does no do anything so far
+		cluster.killTopology(topologyId);
+		cluster.shutdown();
+	}
+
+}


[12/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/api/SplitStreamTypeKeySelector.java
----------------------------------------------------------------------
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 30227b8..0000000
--- 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<SplitStreamType<Tuple>, Tuple> {
-	private static final long serialVersionUID = 4672434660037669254L;
-
-	private final ArrayKeySelector<Tuple> selector;
-
-	public SplitStreamTypeKeySelector(int... fields) {
-		this.selector = new KeySelectorUtil.ArrayKeySelector<Tuple>(fields);
-	}
-
-	@Override
-	public Tuple getKey(SplitStreamType<Tuple> value) throws Exception {
-		return selector.getKey(value.value);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java
deleted file mode 100644
index 114fa7c..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FiniteStormSpout.java
+++ /dev/null
@@ -1,39 +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.util;
-
-import org.apache.flink.stormcompatibility.wrappers.FiniteStormSpoutWrapper;
-
-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}.
- */
-public interface FiniteStormSpout extends IRichSpout {
-
-	/**
-	 * When returns true, the spout has reached the end of the stream.
-	 *
-	 * @return true, if the spout's stream reached its end, false otherwise
-	 */
-	public boolean reachedEnd();
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java
deleted file mode 100644
index 3eee8d6..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkOutputFieldsDeclarer.java
+++ /dev/null
@@ -1,168 +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.util;
-
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.utils.Utils;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-
-import java.util.HashMap;
-import java.util.List;
-
-/**
- * {@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}.<br />
- * <br />
- * <strong>CAUTION: Flink does not support direct emit.</strong>
- */
-public final class FlinkOutputFieldsDeclarer implements OutputFieldsDeclarer {
-
-	/** The declared output streams and schemas. */
-	public final HashMap<String, Fields> outputStreams = new HashMap<String, Fields>();
-
-	@Override
-	public void declare(final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
-	}
-
-	/**
-	 * {@inheritDoc}
-	 * <p/>
-	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		if {@code direct} is {@code true}
-	 */
-	@Override
-	public void declare(final boolean direct, final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
-	}
-
-	@Override
-	public void declareStream(final String streamId, final Fields fields) {
-		this.declareStream(streamId, false, fields);
-	}
-
-	/**
-	 * {@inheritDoc}
-	 * <p/>
-	 * Direct emit is no supported by Flink. Parameter {@code direct} must be {@code false}.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		if {@code direct} is {@code true}
-	 */
-	@Override
-	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
-		if (direct) {
-			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
-		}
-
-		this.outputStreams.put(streamId, fields);
-	}
-
-	/**
-	 * Returns {@link TypeInformation} for the declared output schema for a specific stream.
-	 * 
-	 * @param streamId
-	 *            A stream ID.
-	 * 
-	 * @return output type information for the declared output schema of the specified stream; or {@code null} if
-	 *         {@code streamId == null}
-	 * 
-	 * @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 {
-		if (streamId == null) {
-			return null;
-		}
-
-		Fields outputSchema = this.outputStreams.get(streamId);
-		if (outputSchema == null) {
-			throw new IllegalArgumentException("Stream with ID '" + streamId
-					+ "' was not declared.");
-		}
-
-		Tuple t;
-		final int numberOfAttributes = outputSchema.size();
-
-		if (numberOfAttributes == 1) {
-			return TypeExtractor.getForClass(Object.class);
-		} else if (numberOfAttributes <= 25) {
-			try {
-				t = Tuple.getTupleClass(numberOfAttributes).newInstance();
-			} catch (final InstantiationException e) {
-				throw new RuntimeException(e);
-			} catch (final IllegalAccessException e) {
-				throw new RuntimeException(e);
-			}
-		} else {
-			throw new IllegalArgumentException("Flink supports only a maximum number of 25 attributes");
-		}
-
-		// TODO: declare only key fields as DefaultComparable
-		for (int i = 0; i < numberOfAttributes; ++i) {
-			t.setField(new DefaultComparable(), i);
-		}
-
-		return TypeExtractor.getForObject(t);
-	}
-
-	/**
-	 * {@link DefaultComparable} is a {@link Comparable} helper class that is used to get the correct {@link
-	 * TypeInformation} from {@link TypeExtractor} within {@link #getOutputType()}. If key fields are not comparable,
-	 * Flink cannot use them and will throw an exception.
-	 */
-	private static class DefaultComparable implements Comparable<DefaultComparable> {
-
-		public DefaultComparable() {
-		}
-
-		@Override
-		public int compareTo(final DefaultComparable o) {
-			return 0;
-		}
-	}
-
-	/**
-	 * Computes the indexes within the declared output schema of the specified stream, for a list of given
-	 * field-grouping attributes.
-	 * 
-	 * @param streamId
-	 *            A stream ID.
-	 * @param groupingFields
-	 *            The names of the key fields.
-	 * 
-	 * @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<String> groupingFields) {
-		final int[] fieldIndexes = new int[groupingFields.size()];
-
-		for (int i = 0; i < fieldIndexes.length; ++i) {
-			fieldIndexes[i] = this.outputStreams.get(streamId).fieldIndex(groupingFields.get(i));
-		}
-
-		return fieldIndexes;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 7e60a87..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkStormStreamSelector.java
+++ /dev/null
@@ -1,48 +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.util;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.flink.stormcompatibility.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<T> implements OutputSelector<SplitStreamType<T>> {
-	private static final long serialVersionUID = 2553423379715401023L;
-
-	/** internal cache to avoid short living ArrayList objects. */
-	private final HashMap<String, List<String>> streams = new HashMap<String, List<String>>();
-
-	@Override
-	public Iterable<String> select(SplitStreamType<T> value) {
-		String sid = value.streamId;
-		List<String> streamId = this.streams.get(sid);
-		if (streamId == null) {
-			streamId = new ArrayList<String>(1);
-			streamId.add(sid);
-			this.streams.put(sid, streamId);
-		}
-		return streamId;
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java
deleted file mode 100644
index 14af830..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/FlinkTopologyContext.java
+++ /dev/null
@@ -1,164 +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.util;
-
-import backtype.storm.generated.StormTopology;
-import backtype.storm.hooks.ITaskHook;
-import backtype.storm.metric.api.CombinedMetric;
-import backtype.storm.metric.api.ICombiner;
-import backtype.storm.metric.api.IMetric;
-import backtype.storm.metric.api.IReducer;
-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 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,
-			final Map<Integer, String> taskToComponent, final Map<String, List<Integer>> componentToSortedTasks,
-			final Map<String, Map<String, Fields>> componentToStreamToFields, final String stormId, final String codeDir,
-			final String pidDir, final Integer taskId, final Integer workerPort, final List<Integer> workerTasks,
-			final Map<String, Object> defaultResources, final Map<String, Object> userResources,
-			final Map<String, Object> 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);
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public void addTaskHook(final ITaskHook hook) {
-		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public Collection<ITaskHook> getHooks() {
-		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public IMetric getRegisteredMetricByName(final String name) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@SuppressWarnings("rawtypes")
-	@Override
-	public CombinedMetric registerMetric(final String name, final ICombiner combiner, final int timeBucketSizeInSecs) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@SuppressWarnings("rawtypes")
-	@Override
-	public ReducedMetric registerMetric(final String name, final IReducer combiner, final int timeBucketSizeInSecs) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@SuppressWarnings("unchecked")
-	@Override
-	public IMetric registerMetric(final String name, final IMetric metric, final int timeBucketSizeInSecs) {
-		throw new UnsupportedOperationException("Metrics are not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public <T extends ISubscribedState> T setAllSubscribedState(final T obj) {
-		throw new UnsupportedOperationException("Not supported by Flink");
-
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final T obj) {
-		throw new UnsupportedOperationException("Not supported by Flink");
-	}
-
-	/**
-	 * Not supported by Flink.
-	 *
-	 * @throws UnsupportedOperationException
-	 * 		at every invocation
-	 */
-	@Override
-	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final String streamId, final T
-			obj) {
-		throw new UnsupportedOperationException("Not supported by Flink");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java
deleted file mode 100644
index 9cb44ec..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamMapper.java
+++ /dev/null
@@ -1,39 +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.util;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.SplitStream;
-
-/**
- * Strips {@link SplitStreamType}{@code <T>} away, ie, extracts the wrapped record of type {@code T}. Can be used to get
- * a "clean" stream from a Spout/Bolt that declared multiple output streams (after the streams got separated using
- * {@link DataStream#split(org.apache.flink.streaming.api.collector.selector.OutputSelector) .split(...)} and
- * {@link SplitStream#select(String...) .select(...)}).
- * 
- * @param <T>
- */
-public class SplitStreamMapper<T> implements MapFunction<SplitStreamType<T>, T> {
-	private static final long serialVersionUID = 3550359150160908564L;
-
-	@Override
-	public T map(SplitStreamType<T> value) throws Exception {
-		return value.value;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java
deleted file mode 100644
index 9c7e477..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamType.java
+++ /dev/null
@@ -1,52 +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.util;
-
-import org.apache.flink.streaming.api.datastream.DataStream;
-
-/**
- * Used by {@link org.apache.flink.stormcompatibility.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}.
- */
-public class SplitStreamType<T> {
-
-	/** The stream ID this tuple belongs to. */
-	public String streamId;
-	/** The actual data value. */
-	public T value;
-
-	@Override
-	public String toString() {
-		return "<sid:" + this.streamId + ",v:" + this.value + ">";
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-		SplitStreamType<?> other = (SplitStreamType<?>) o;
-
-		return this.streamId.equals(other.streamId) && this.value.equals(other.value);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 200f772..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/SplitStreamTypeKeySelector.java
+++ /dev/null
@@ -1,46 +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.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<SplitStreamType<Tuple>, Tuple> {
-	private static final long serialVersionUID = 4672434660037669254L;
-
-	private final ArrayKeySelector<Tuple> selector;
-
-	public SplitStreamTypeKeySelector(int... fields) {
-		this.selector = new KeySelectorUtil.ArrayKeySelector<Tuple>(fields);
-	}
-
-	@Override
-	public Tuple getKey(SplitStreamType<Tuple> value) throws Exception {
-		return selector.getKey(value.value);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/StormConfig.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/StormConfig.java
deleted file mode 100644
index 6726ae8..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/util/StormConfig.java
+++ /dev/null
@@ -1,123 +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.util;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters;
-
-import backtype.storm.Config;
-
-/**
- * {@link StormConfig} is used to provide a user-defined Storm configuration (ie, a raw {@link Map} or {@link Config}
- * object) for embedded Spouts and Bolts.
- */
-@SuppressWarnings("rawtypes")
-public final class StormConfig extends GlobalJobParameters implements Map {
-	private static final long serialVersionUID = 8019519109673698490L;
-
-	/** Contains the actual configuration that is provided to Spouts and Bolts. */
-	private final Map config = new HashMap();
-
-	/**
-	 * Creates an empty configuration.
-	 */
-	public StormConfig() {
-	}
-
-	/**
-	 * Creates an configuration with initial values provided by the given {@code Map}.
-	 * 
-	 * @param config
-	 *            Initial values for this configuration.
-	 */
-	@SuppressWarnings("unchecked")
-	public StormConfig(Map config) {
-		this.config.putAll(config);
-	}
-
-
-	@Override
-	public int size() {
-		return this.config.size();
-	}
-
-	@Override
-	public boolean isEmpty() {
-		return this.config.isEmpty();
-	}
-
-	@Override
-	public boolean containsKey(Object key) {
-		return this.config.containsKey(key);
-	}
-
-	@Override
-	public boolean containsValue(Object value) {
-		return this.config.containsValue(value);
-	}
-
-	@Override
-	public Object get(Object key) {
-		return this.config.get(key);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public Object put(Object key, Object value) {
-		return this.config.put(key, value);
-	}
-
-	@Override
-	public Object remove(Object key) {
-		return this.config.remove(key);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void putAll(Map m) {
-		this.config.putAll(m);
-	}
-
-	@Override
-	public void clear() {
-		this.config.clear();
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public Set<Object> keySet() {
-		return this.config.keySet();
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public Collection<Object> values() {
-		return this.config.values();
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public Set<java.util.Map.Entry<Object, Object>> entrySet() {
-		return this.config.entrySet();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java
deleted file mode 100644
index 7b35a64..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormCollector.java
+++ /dev/null
@@ -1,143 +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.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 java.util.HashMap;
-import java.util.List;
-import java.util.Map.Entry;
-
-/**
- * A {@link AbstractStormCollector} transforms Storm tuples to Flink tuples.
- */
-abstract class AbstractStormCollector<OUT> {
-
-	/** Flink output tuple of concrete type {@link Tuple0} to {@link Tuple25} per output stream. */
-	protected final HashMap<String, Tuple> outputTuple = new HashMap<String, Tuple>();
-	/** Flink split tuple. Used, if multiple output streams are declared. */
-	private final SplitStreamType<Object> splitTuple = new SplitStreamType<Object>();
-	/**
-	 * The number of attributes of the output tuples per stream. (Determines the concrete type of {@link #outputTuple}).
-	 * If {@link #numberOfAttributes} is zero, {@link #outputTuple} is not used and "raw" data type is used.
-	 */
-	protected final HashMap<String, Integer> numberOfAttributes;
-	/** Indicates of multiple output stream are declared and thus {@link SplitStreamType} must be used as output. */
-	private final boolean split;
-	/** Is set to {@code true} each time a tuple is emitted. */
-	boolean tupleEmitted = false;
-
-	/**
-	 * Instantiates a new {@link AbstractStormCollector} that emits Flink tuples via {@link #doEmit(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.
-	 * 
-	 * @param numberOfAttributes
-	 *            The number of attributes of the emitted tuples per output stream.
-	 * @throws UnsupportedOperationException
-	 *             if the specified number of attributes is greater than 25
-	 */
-	public AbstractStormCollector(final HashMap<String, Integer> numberOfAttributes)
-			throws UnsupportedOperationException {
-		assert (numberOfAttributes != null);
-
-		this.numberOfAttributes = numberOfAttributes;
-		this.split = this.numberOfAttributes.size() > 1;
-
-		for (Entry<String, Integer> outputStream : numberOfAttributes.entrySet()) {
-			final int numAtt = outputStream.getValue();
-			assert (numAtt >= -1);
-
-			if (numAtt > 25) {
-				throw new UnsupportedOperationException(
-						"Flink cannot handle more then 25 attributes, but " + numAtt
-						+ " are declared for stream '" + outputStream.getKey()
-						+ "' by the given bolt");
-			} else if (numAtt >= 0) {
-				try {
-					this.outputTuple.put(outputStream.getKey(),
-							org.apache.flink.api.java.tuple.Tuple.getTupleClass(numAtt)
-							.newInstance());
-				} catch (final InstantiationException e) {
-					throw new RuntimeException(e);
-				} catch (final IllegalAccessException e) {
-					throw new RuntimeException(e);
-				}
-
-			}
-		}
-	}
-
-	/**
-	 * Transforms a Storm tuple into a Flink tuple of type {@code OUT} and emits this tuple via {@link #doEmit(Object)}
-	 * to the specified output stream.
-	 * 
-	 * @param The
-	 *            The output stream id.
-	 * @param tuple
-	 *            The Storm tuple to be emitted.
-	 * @return the return value of {@link #doEmit(Object)}
-	 */
-	@SuppressWarnings("unchecked")
-	protected final List<Integer> tansformAndEmit(final String streamId, final List<Object> tuple) {
-		List<Integer> taskIds;
-
-		final int numAtt = this.numberOfAttributes.get(streamId);
-		if (numAtt > -1) {
-			assert (tuple.size() == numAtt);
-			Tuple out = this.outputTuple.get(streamId);
-			for (int i = 0; i < numAtt; ++i) {
-				out.setField(tuple.get(i), i);
-			}
-			if (this.split) {
-				this.splitTuple.streamId = streamId;
-				this.splitTuple.value = out;
-
-				taskIds = doEmit((OUT) this.splitTuple);
-			} else {
-				taskIds = doEmit((OUT) out);
-			}
-
-		} else {
-			assert (tuple.size() == 1);
-			if (split) {
-				this.splitTuple.streamId = streamId;
-				this.splitTuple.value = tuple.get(0);
-
-				taskIds = doEmit((OUT) this.splitTuple);
-			} else {
-				taskIds = doEmit((OUT) tuple.get(0));
-			}
-		}
-		this.tupleEmitted = true;
-
-		return taskIds;
-	}
-
-	/**
-	 * Emits a Flink tuple.
-	 * 
-	 * @param flinkTuple
-	 * 		The tuple to be emitted.
-	 * @return the IDs of the tasks this tuple was sent to
-	 */
-	protected abstract List<Integer> doEmit(OUT flinkTuple);
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/AbstractStormSpoutWrapper.java
----------------------------------------------------------------------
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 ccd29bb..0000000
--- 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).<br />
- * <br />
- * <strong>CAUTION: currently, only simple spouts are supported! (ie, spouts that do not use the Storm configuration
- * <code>Map</code> or <code>TopologyContext</code> that is provided by the spouts's <code>prepare(..)</code> method.
- * Furthermore, ack and fail back calls as well as tuple IDs are not supported so far.</strong>
- */
-public abstract class AbstractStormSpoutWrapper<OUT> extends RichParallelSourceFunction<OUT> {
-	private static final long serialVersionUID = 4993283609095408765L;
-
-	/** Number of attributes of the bolt's output tuples per stream. */
-	private final HashMap<String, Integer> numberOfAttributes;
-	/** The wrapped Storm {@link IRichSpout spout}. */
-	protected final IRichSpout spout;
-	/** The wrapper of the given Flink collector. */
-	protected StormSpoutCollector<OUT> 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<String> 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<OUT> ctx) throws Exception {
-		this.collector = new StormSpoutCollector<OUT>(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}
-	 * <p/>
-	 * Sets the {@link #isRunning} flag to {@code false}.
-	 */
-	@Override
-	public void cancel() {
-		this.isRunning = false;
-	}
-
-	@Override
-	public void close() throws Exception {
-		this.spout.close();
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/FiniteStormSpoutWrapper.java
----------------------------------------------------------------------
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 f499ecc..0000000
--- 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<OUT> extends AbstractStormSpoutWrapper<OUT> {
-	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<String> 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();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java
deleted file mode 100644
index 3cd27d4..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/SetupOutputFieldsDeclarer.java
+++ /dev/null
@@ -1,63 +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.HashMap;
-
-import backtype.storm.topology.OutputFieldsDeclarer;
-import backtype.storm.tuple.Fields;
-import backtype.storm.utils.Utils;
-
-/**
- * {@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 SetupOutputFieldsDeclarer implements OutputFieldsDeclarer {
-
-	/** The number of attributes for each declared stream by the wrapped operator. */
-	HashMap<String, Integer> outputSchemas = new HashMap<String, Integer>();
-
-	@Override
-	public void declare(final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
-	}
-
-	@Override
-	public void declare(final boolean direct, final Fields fields) {
-		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
-	}
-
-	@Override
-	public void declareStream(final String streamId, final Fields fields) {
-		this.declareStream(streamId, false, fields);
-	}
-
-	@Override
-	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
-		if (streamId == null) {
-			throw new IllegalArgumentException("Stream ID cannot be null.");
-		}
-		if (direct) {
-			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
-		}
-
-		this.outputSchemas.put(streamId, fields.size());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java
deleted file mode 100644
index e810214..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltCollector.java
+++ /dev/null
@@ -1,94 +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.task.IOutputCollector;
-import backtype.storm.tuple.Tuple;
-
-import org.apache.flink.api.java.tuple.Tuple0;
-import org.apache.flink.api.java.tuple.Tuple25;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.util.Collector;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-
-/**
- * A {@link StormBoltCollector} is used by {@link StormBoltWrapper} 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<OUT> extends AbstractStormCollector<OUT> implements IOutputCollector {
-
-	/** The Flink output Collector */
-	private final Collector<OUT> flinkOutput;
-
-	/**
-	 * Instantiates a new {@link StormBoltCollector} 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.
-	 * 
-	 * @param numberOfAttributes
-	 *            The number of attributes of the emitted tuples per output stream.
-	 * @param flinkOutput
-	 *            The Flink output object to be used.
-	 * @throws UnsupportedOperationException
-	 *             if the specified number of attributes is greater than 25
-	 */
-	public StormBoltCollector(final HashMap<String, Integer> numberOfAttributes,
-			final Collector<OUT> flinkOutput) throws UnsupportedOperationException {
-		super(numberOfAttributes);
-		assert (flinkOutput != null);
-		this.flinkOutput = flinkOutput;
-	}
-
-	@Override
-	protected List<Integer> doEmit(final OUT flinkTuple) {
-		this.flinkOutput.collect(flinkTuple);
-		// TODO
-		return null;
-	}
-
-	@Override
-	public void reportError(final Throwable error) {
-		// not sure, if Flink can support this
-		throw new UnsupportedOperationException("Not implemented yet");
-	}
-
-	@Override
-	public List<Integer> emit(final String streamId, final Collection<Tuple> anchors, final List<Object> tuple) {
-		return this.tansformAndEmit(streamId, tuple);
-	}
-
-	@Override
-	public void emitDirect(final int taskId, final String streamId, final Collection<Tuple> anchors, final List<Object> tuple) {
-		throw new UnsupportedOperationException("Direct emit is not supported by Flink");
-	}
-
-	@Override
-	public void ack(final Tuple input) {
-		throw new UnsupportedOperationException("Currently, acking/failing is not supported by Flink");
-	}
-
-	@Override
-	public void fail(final Tuple input) {
-		throw new UnsupportedOperationException("Currently, acking/failing is not supported by Flink");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 715d6df..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormBoltWrapper.java
+++ /dev/null
@@ -1,260 +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.task.OutputCollector;
-import backtype.storm.task.TopologyContext;
-import backtype.storm.topology.IRichBolt;
-import backtype.storm.tuple.Fields;
-
-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.configuration.Configuration;
-import org.apache.flink.stormcompatibility.util.SplitStreamType;
-import org.apache.flink.stormcompatibility.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;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-import com.google.common.collect.Sets;
-
-/**
- * A {@link StormBoltWrapper} 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).<br />
- * <br />
- * <strong>CAUTION: currently, only simple bolts are supported! (ie, bolts that do not use the Storm configuration
- * <code>Map</code> or <code>TopologyContext</code> that is provided by the bolt's <code>open(..)</code> method.
- * Furthermore, acking and failing of tuples as well as accessing tuple attributes by field names is not supported so
- * far.</strong>
- */
-public class StormBoltWrapper<IN, OUT> extends AbstractStreamOperator<OUT> implements OneInputStreamOperator<IN, OUT> {
-	private static final long serialVersionUID = -4788589118464155835L;
-
-	/** The wrapped Storm {@link IRichBolt bolt}. */
-	private final IRichBolt bolt;
-	/** Number of attributes of the bolt's output tuples per stream. */
-	private final HashMap<String, Integer> 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
-	 *  {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}.
-	 */
-	private TimestampedCollector<OUT> flinkCollector;
-
-	/**
-	 * Instantiates a new {@link StormBoltWrapper} 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.
-	 * 
-	 * @param bolt
-	 *            The Storm {@link IRichBolt bolt} to be used.
-	 * @throws IllegalArgumentException
-	 *             If the number of declared output attributes is not with range [0;25].
-	 */
-	public StormBoltWrapper(final IRichBolt bolt) throws IllegalArgumentException {
-		this(bolt, null, (Collection<String>) null);
-	}
-
-	/**
-	 * Instantiates a new {@link StormBoltWrapper} 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.
-	 * 
-	 * @param bolt
-	 *            The Storm {@link IRichBolt bolt} to be used.
-	 * @param inputSchema
-	 *            The schema (ie, ordered field names) of the input stream.
-	 * @throws IllegalArgumentException
-	 *             If the number of declared output attributes is not with range [0;25].
-	 */
-	public StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema)
-			throws IllegalArgumentException {
-		this(bolt, inputSchema, (Collection<String>) null);
-	}
-
-	/**
-	 * Instantiates a new {@link StormBoltWrapper} 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
-	 * of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
-	 * 
-	 * @param bolt
-	 *            The Storm {@link IRichBolt bolt} 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
-	 *             [1;25].
-	 */
-	public StormBoltWrapper(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
-	 * 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
-	 * of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
-	 * 
-	 * @param bolt
-	 *            The Storm {@link IRichBolt bolt} 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
-	 *             [1;25].
-	 */
-	public StormBoltWrapper(final IRichBolt bolt, final Collection<String> 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
-	 * 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
-	 * be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
-	 * 
-	 * @param bolt
-	 *            The Storm {@link IRichBolt bolt} to be used.
-	 * @param inputSchema
-	 *            The schema (ie, ordered field names) of the input stream.
-	 * @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 StormBoltWrapper(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
-	 * 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
-	 * be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
-	 * 
-	 * @param bolt
-	 *            The Storm {@link IRichBolt bolt} to be used.
-	 * @param inputSchema
-	 *            The schema (ie, ordered field names) of the input stream.
-	 * @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 StormBoltWrapper(final IRichBolt bolt, final Fields inputSchema,
-			final Collection<String> rawOutputs) throws IllegalArgumentException {
-		this.bolt = bolt;
-		this.inputSchema = 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.createTopologyContext(
-				super.runtimeContext, this.bolt, this.stormTopology, null);
-		flinkCollector = new TimestampedCollector<OUT>(output);
-		OutputCollector stormCollector = null;
-
-		if (this.numberOfAttributes.size() > 0) {
-			stormCollector = new OutputCollector(new StormBoltCollector<OUT>(
-					this.numberOfAttributes, flinkCollector));
-		}
-
-		GlobalJobParameters config = super.executionConfig.getGlobalJobParameters();
-		StormConfig stormConfig = new StormConfig();
-
-		if (config != null) {
-			if (config instanceof StormConfig) {
-				stormConfig = (StormConfig) config;
-			} else {
-				stormConfig.putAll(config.toMap());
-			}
-		}
-
-		this.bolt.prepare(stormConfig, topologyContext, stormCollector);
-	}
-
-	@Override
-	public void dispose() {
-		this.bolt.cleanup();
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void processElement(final StreamRecord<IN> element) throws Exception {
-		flinkCollector.setTimestamp(element.getTimestamp());
-		IN value = element.getValue();
-		if (value instanceof SplitStreamType) {
-			this.bolt.execute(new StormTuple<IN>(((SplitStreamType<IN>) value).value,
-					inputSchema));
-		} else {
-			this.bolt.execute(new StormTuple<IN>(value, inputSchema));
-		}
-	}
-
-	@Override
-	public void processWatermark(Watermark mark) throws Exception {
-		output.emitWatermark(mark);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormFiniteSpoutWrapper.java
----------------------------------------------------------------------
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 45eb56c..0000000
--- 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<OUT> extends AbstractStormSpoutWrapper<OUT> {
-	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<String>) 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<String>) 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<String> 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<String> 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);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java
----------------------------------------------------------------------
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-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java
deleted file mode 100644
index 5a20056..0000000
--- a/flink-contrib/flink-storm-compatibility/flink-storm-compatibility-core/src/main/java/org/apache/flink/stormcompatibility/wrappers/StormSpoutCollector.java
+++ /dev/null
@@ -1,82 +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.ISpoutOutputCollector;
-
-import org.apache.flink.api.java.tuple.Tuple0;
-import org.apache.flink.api.java.tuple.Tuple25;
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-
-import java.util.HashMap;
-import java.util.List;
-
-/**
- * A {@link StormSpoutCollector} 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<OUT> extends AbstractStormCollector<OUT> implements ISpoutOutputCollector {
-
-	/** The Flink source context object */
-	private final SourceContext<OUT> flinkContext;
-
-	/**
-	 * Instantiates a new {@link StormSpoutCollector} 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.
-	 * 
-	 * @param numberOfAttributes
-	 *            The number of attributes of the emitted tuples.
-	 * @param flinkContext
-	 *            The Flink source context to be used.
-	 * @throws UnsupportedOperationException
-	 *             if the specified number of attributes is greater than 25
-	 */
-	public StormSpoutCollector(final HashMap<String, Integer> numberOfAttributes,
-			final SourceContext<OUT> flinkContext) throws UnsupportedOperationException {
-		super(numberOfAttributes);
-		assert (flinkContext != null);
-		this.flinkContext = flinkContext;
-	}
-
-	@Override
-	protected List<Integer> doEmit(final OUT flinkTuple) {
-		this.flinkContext.collect(flinkTuple);
-		// TODO
-		return null;
-	}
-
-	@Override
-	public void reportError(final Throwable error) {
-		// not sure, if Flink can support this
-		throw new UnsupportedOperationException("Not implemented yet");
-	}
-
-	@Override
-	public List<Integer> emit(final String streamId, final List<Object> tuple, final Object messageId) {
-		return this.tansformAndEmit(streamId, tuple);
-	}
-
-
-	@Override
-	public void emitDirect(final int taskId, final String streamId, final List<Object> tuple, final Object messageId) {
-		throw new UnsupportedOperationException("Direct emit is not supported by Flink");
-	}
-
-}


[03/15] flink git commit: [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" *

Posted by mj...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/AbstractStormCollector.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/AbstractStormCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/AbstractStormCollector.java
new file mode 100644
index 0000000..85d895c
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/AbstractStormCollector.java
@@ -0,0 +1,143 @@
+/*
+ * 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 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.storm.util.SplitStreamType;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+
+/**
+ * A {@link AbstractStormCollector} transforms Storm tuples to Flink tuples.
+ */
+abstract class AbstractStormCollector<OUT> {
+
+	/** Flink output tuple of concrete type {@link Tuple0} to {@link Tuple25} per output stream. */
+	protected final HashMap<String, Tuple> outputTuple = new HashMap<String, Tuple>();
+	/** Flink split tuple. Used, if multiple output streams are declared. */
+	private final SplitStreamType<Object> splitTuple = new SplitStreamType<Object>();
+	/**
+	 * The number of attributes of the output tuples per stream. (Determines the concrete type of {@link #outputTuple}).
+	 * If {@link #numberOfAttributes} is zero, {@link #outputTuple} is not used and "raw" data type is used.
+	 */
+	protected final HashMap<String, Integer> numberOfAttributes;
+	/** Indicates of multiple output stream are declared and thus {@link SplitStreamType} must be used as output. */
+	private final boolean split;
+	/** Is set to {@code true} each time a tuple is emitted. */
+	boolean tupleEmitted = false;
+
+	/**
+	 * Instantiates a new {@link AbstractStormCollector} that emits Flink tuples via {@link #doEmit(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.
+	 * 
+	 * @param numberOfAttributes
+	 *            The number of attributes of the emitted tuples per output stream.
+	 * @throws UnsupportedOperationException
+	 *             if the specified number of attributes is greater than 25
+	 */
+	AbstractStormCollector(final HashMap<String, Integer> numberOfAttributes)
+			throws UnsupportedOperationException {
+		assert (numberOfAttributes != null);
+
+		this.numberOfAttributes = numberOfAttributes;
+		this.split = this.numberOfAttributes.size() > 1;
+
+		for (Entry<String, Integer> outputStream : numberOfAttributes.entrySet()) {
+			final int numAtt = outputStream.getValue();
+			assert (numAtt >= -1);
+
+			if (numAtt > 25) {
+				throw new UnsupportedOperationException(
+						"Flink cannot handle more then 25 attributes, but " + numAtt
+						+ " are declared for stream '" + outputStream.getKey()
+						+ "' by the given bolt");
+			} else if (numAtt >= 0) {
+				try {
+					this.outputTuple.put(outputStream.getKey(),
+							org.apache.flink.api.java.tuple.Tuple.getTupleClass(numAtt)
+							.newInstance());
+				} catch (final InstantiationException e) {
+					throw new RuntimeException(e);
+				} catch (final IllegalAccessException e) {
+					throw new RuntimeException(e);
+				}
+
+			}
+		}
+	}
+
+	/**
+	 * Transforms a Storm tuple into a Flink tuple of type {@code OUT} and emits this tuple via {@link #doEmit(Object)}
+	 * to the specified output stream.
+	 * 
+	 * @param The
+	 *            The output stream id.
+	 * @param tuple
+	 *            The Storm tuple to be emitted.
+	 * @return the return value of {@link #doEmit(Object)}
+	 */
+	@SuppressWarnings("unchecked")
+	protected final List<Integer> tansformAndEmit(final String streamId, final List<Object> tuple) {
+		List<Integer> taskIds;
+
+		final int numAtt = this.numberOfAttributes.get(streamId);
+		if (numAtt > -1) {
+			assert (tuple.size() == numAtt);
+			Tuple out = this.outputTuple.get(streamId);
+			for (int i = 0; i < numAtt; ++i) {
+				out.setField(tuple.get(i), i);
+			}
+			if (this.split) {
+				this.splitTuple.streamId = streamId;
+				this.splitTuple.value = out;
+
+				taskIds = doEmit((OUT) this.splitTuple);
+			} else {
+				taskIds = doEmit((OUT) out);
+			}
+
+		} else {
+			assert (tuple.size() == 1);
+			if (split) {
+				this.splitTuple.streamId = streamId;
+				this.splitTuple.value = tuple.get(0);
+
+				taskIds = doEmit((OUT) this.splitTuple);
+			} else {
+				taskIds = doEmit((OUT) tuple.get(0));
+			}
+		}
+		this.tupleEmitted = true;
+
+		return taskIds;
+	}
+
+	/**
+	 * Emits a Flink tuple.
+	 * 
+	 * @param flinkTuple
+	 * 		The tuple to be emitted.
+	 * @return the IDs of the tasks this tuple was sent to
+	 */
+	protected abstract List<Integer> doEmit(OUT flinkTuple);
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java
new file mode 100644
index 0000000..58fd098
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java
@@ -0,0 +1,89 @@
+/*
+ * 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.task.IOutputCollector;
+import backtype.storm.tuple.Tuple;
+
+import org.apache.flink.api.java.tuple.Tuple0;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.util.Collector;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * 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 BoltCollector<OUT> extends AbstractStormCollector<OUT> implements IOutputCollector {
+
+	/** The Flink output Collector */
+	private final Collector<OUT> flinkOutput;
+
+	/**
+	 * 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.
+	 * 
+	 * @param numberOfAttributes
+	 *            The number of attributes of the emitted tuples per output stream.
+	 * @param flinkOutput
+	 *            The Flink output object to be used.
+	 * @throws UnsupportedOperationException
+	 *             if the specified number of attributes is greater than 25
+	 */
+	BoltCollector(final HashMap<String, Integer> numberOfAttributes,
+			final Collector<OUT> flinkOutput) throws UnsupportedOperationException {
+		super(numberOfAttributes);
+		assert (flinkOutput != null);
+		this.flinkOutput = flinkOutput;
+	}
+
+	@Override
+	protected List<Integer> doEmit(final OUT flinkTuple) {
+		this.flinkOutput.collect(flinkTuple);
+		// TODO
+		return null;
+	}
+
+	@Override
+	public void reportError(final Throwable error) {
+		// not sure, if Flink can support this
+	}
+
+	@Override
+	public List<Integer> emit(final String streamId, final Collection<Tuple> anchors, final List<Object> tuple) {
+		return this.tansformAndEmit(streamId, tuple);
+	}
+
+	@Override
+	public void emitDirect(final int taskId, final String streamId, final Collection<Tuple> anchors, final List<Object> tuple) {
+		throw new UnsupportedOperationException("Direct emit is not supported by Flink");
+	}
+
+	@Override
+	public void ack(final Tuple input) {}
+
+	@Override
+	public void fail(final Tuple input) {}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
new file mode 100644
index 0000000..b16fc09
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
@@ -0,0 +1,260 @@
+/*
+ * 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.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.tuple.Fields;
+
+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.configuration.Configuration;
+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;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import com.google.common.collect.Sets;
+
+/**
+ * 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).<br />
+ * <br />
+ * <strong>CAUTION: currently, only simple bolts are supported! (ie, bolts that do not use the Storm configuration
+ * <code>Map</code> or <code>TopologyContext</code> that is provided by the bolt's <code>open(..)</code> method.
+ * Furthermore, acking and failing of tuples as well as accessing tuple attributes by field names is not supported so
+ * far.</strong>
+ */
+public class BoltWrapper<IN, OUT> extends AbstractStreamOperator<OUT> implements OneInputStreamOperator<IN, OUT> {
+	private static final long serialVersionUID = -4788589118464155835L;
+
+	/** The wrapped Storm {@link IRichBolt bolt}. */
+	private final IRichBolt bolt;
+	/** Number of attributes of the bolt's output tuples per stream. */
+	private final HashMap<String, Integer> 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
+	 *  {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}.
+	 */
+	private TimestampedCollector<OUT> flinkCollector;
+
+	/**
+	 * 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.
+	 * 
+	 * @param bolt
+	 *            The Storm {@link IRichBolt bolt} to be used.
+	 * @throws IllegalArgumentException
+	 *             If the number of declared output attributes is not with range [0;25].
+	 */
+	public BoltWrapper(final IRichBolt bolt) throws IllegalArgumentException {
+		this(bolt, null, (Collection<String>) null);
+	}
+
+	/**
+	 * 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.
+	 * 
+	 * @param bolt
+	 *            The Storm {@link IRichBolt bolt} to be used.
+	 * @param inputSchema
+	 *            The schema (ie, ordered field names) of the input stream.
+	 * @throws IllegalArgumentException
+	 *             If the number of declared output attributes is not with range [0;25].
+	 */
+	public BoltWrapper(final IRichBolt bolt, final Fields inputSchema)
+			throws IllegalArgumentException {
+		this(bolt, inputSchema, (Collection<String>) null);
+	}
+
+	/**
+	 * 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
+	 * of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
+	 * 
+	 * @param bolt
+	 *            The Storm {@link IRichBolt bolt} 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
+	 *             [1;25].
+	 */
+	public BoltWrapper(final IRichBolt bolt, final String[] rawOutputs)
+			throws IllegalArgumentException {
+		this(bolt, null, Sets.newHashSet(rawOutputs));
+	}
+
+	/**
+	 * 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
+	 * of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
+	 * 
+	 * @param bolt
+	 *            The Storm {@link IRichBolt bolt} 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
+	 *             [1;25].
+	 */
+	public BoltWrapper(final IRichBolt bolt, final Collection<String> rawOutputs)
+			throws IllegalArgumentException {
+		this(bolt, null, rawOutputs);
+	}
+
+	/**
+	 * 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
+	 * be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
+	 * 
+	 * @param bolt
+	 *            The Storm {@link IRichBolt bolt} to be used.
+	 * @param inputSchema
+	 *            The schema (ie, ordered field names) of the input stream.
+	 * @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 BoltWrapper(final IRichBolt bolt, final Fields inputSchema,
+			final String[] rawOutputs) throws IllegalArgumentException {
+		this(bolt, inputSchema, Sets.newHashSet(rawOutputs));
+	}
+
+	/**
+	 * 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
+	 * be one of {@link Tuple0} to {@link Tuple25} depending on the bolt's declared number of attributes.
+	 * 
+	 * @param bolt
+	 *            The Storm {@link IRichBolt bolt} to be used.
+	 * @param inputSchema
+	 *            The schema (ie, ordered field names) of the input stream.
+	 * @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 BoltWrapper(final IRichBolt bolt, final Fields inputSchema,
+			final Collection<String> rawOutputs) throws IllegalArgumentException {
+		this.bolt = bolt;
+		this.inputSchema = inputSchema;
+		this.numberOfAttributes = WrapperSetupHelper.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);
+
+		this.flinkCollector = new TimestampedCollector<OUT>(output);
+		OutputCollector stormCollector = null;
+
+		if (this.numberOfAttributes.size() > 0) {
+			stormCollector = new OutputCollector(new BoltCollector<OUT>(
+					this.numberOfAttributes, flinkCollector));
+		}
+
+		GlobalJobParameters config = super.executionConfig.getGlobalJobParameters();
+		StormConfig stormConfig = new StormConfig();
+
+		if (config != null) {
+			if (config instanceof StormConfig) {
+				stormConfig = (StormConfig) config;
+			} else {
+				stormConfig.putAll(config.toMap());
+			}
+		}
+
+		final TopologyContext topologyContext = WrapperSetupHelper.createTopologyContext(
+				super.runtimeContext, this.bolt, this.stormTopology, stormConfig);
+
+		this.bolt.prepare(stormConfig, topologyContext, stormCollector);
+	}
+
+	@Override
+	public void dispose() {
+		this.bolt.cleanup();
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public void processElement(final StreamRecord<IN> element) throws Exception {
+		this.flinkCollector.setTimestamp(element.getTimestamp());
+		IN value = element.getValue();
+		if (value instanceof SplitStreamType) {
+			this.bolt.execute(new StormTuple<IN>(((SplitStreamType<IN>) value).value,
+					inputSchema));
+		} else {
+			this.bolt.execute(new StormTuple<IN>(value, inputSchema));
+		}
+	}
+
+	@Override
+	public void processWatermark(Watermark mark) throws Exception {
+		this.output.emitWatermark(mark);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java
new file mode 100644
index 0000000..68368bf
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java
@@ -0,0 +1,165 @@
+/*
+ * 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.generated.StormTopology;
+import backtype.storm.hooks.ITaskHook;
+import backtype.storm.metric.api.CombinedMetric;
+import backtype.storm.metric.api.ICombiner;
+import backtype.storm.metric.api.IMetric;
+import backtype.storm.metric.api.IReducer;
+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.
+ */
+final class FlinkTopologyContext extends TopologyContext {
+
+	/**
+	 * Instantiates a new {@link FlinkTopologyContext} for a given Storm topology. The context object is instantiated
+	 * for each parallel task
+	 */
+	FlinkTopologyContext(final StormTopology topology,
+			@SuppressWarnings("rawtypes") final Map stormConf,
+			final Map<Integer, String> taskToComponent, final Map<String, List<Integer>> componentToSortedTasks,
+			final Map<String, Map<String, Fields>> componentToStreamToFields, final String stormId, final String codeDir,
+			final String pidDir, final Integer taskId, final Integer workerPort, final List<Integer> workerTasks,
+			final Map<String, Object> defaultResources, final Map<String, Object> userResources,
+			final Map<String, Object> 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);
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public void addTaskHook(final ITaskHook hook) {
+		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public Collection<ITaskHook> getHooks() {
+		throw new UnsupportedOperationException("Task hooks are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public IMetric getRegisteredMetricByName(final String name) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@SuppressWarnings("rawtypes")
+	@Override
+	public CombinedMetric registerMetric(final String name, final ICombiner combiner, final int timeBucketSizeInSecs) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@SuppressWarnings("rawtypes")
+	@Override
+	public ReducedMetric registerMetric(final String name, final IReducer combiner, final int timeBucketSizeInSecs) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@SuppressWarnings("unchecked")
+	@Override
+	public IMetric registerMetric(final String name, final IMetric metric, final int timeBucketSizeInSecs) {
+		throw new UnsupportedOperationException("Metrics are not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public <T extends ISubscribedState> T setAllSubscribedState(final T obj) {
+		throw new UnsupportedOperationException("Not supported by Flink");
+
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final T obj) {
+		throw new UnsupportedOperationException("Not supported by Flink");
+	}
+
+	/**
+	 * Not supported by Flink.
+	 *
+	 * @throws UnsupportedOperationException
+	 * 		at every invocation
+	 */
+	@Override
+	public <T extends ISubscribedState> T setSubscribedState(final String componentId, final String streamId, final T
+			obj) {
+		throw new UnsupportedOperationException("Not supported by Flink");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java
new file mode 100644
index 0000000..507305b
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java
@@ -0,0 +1,66 @@
+/*
+ * 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.HashMap;
+
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Fields;
+import backtype.storm.utils.Utils;
+
+/**
+ * {@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<String, Fields> outputStreams = new HashMap<String, Fields>();
+	/** The number of attributes for each declared stream by the wrapped operator. */
+	HashMap<String, Integer> outputSchemas = new HashMap<String, Integer>();
+
+	@Override
+	public void declare(final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, false, fields);
+	}
+
+	@Override
+	public void declare(final boolean direct, final Fields fields) {
+		this.declareStream(Utils.DEFAULT_STREAM_ID, direct, fields);
+	}
+
+	@Override
+	public void declareStream(final String streamId, final Fields fields) {
+		this.declareStream(streamId, false, fields);
+	}
+
+	@Override
+	public void declareStream(final String streamId, final boolean direct, final Fields fields) {
+		if (streamId == null) {
+			throw new IllegalArgumentException("Stream ID cannot be null.");
+		}
+		if (direct) {
+			throw new UnsupportedOperationException("Direct emit is not supported by Flink");
+		}
+
+		this.outputStreams.put(streamId, fields);
+		this.outputSchemas.put(streamId, fields.size());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java
new file mode 100644
index 0000000..91fc090
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java
@@ -0,0 +1,81 @@
+/*
+ * 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.ISpoutOutputCollector;
+
+import org.apache.flink.api.java.tuple.Tuple0;
+import org.apache.flink.api.java.tuple.Tuple25;
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
+
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * 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 SpoutCollector<OUT> extends AbstractStormCollector<OUT> implements ISpoutOutputCollector {
+
+	/** The Flink source context object */
+	private final SourceContext<OUT> flinkContext;
+
+	/**
+	 * 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.
+	 * 
+	 * @param numberOfAttributes
+	 *            The number of attributes of the emitted tuples.
+	 * @param flinkContext
+	 *            The Flink source context to be used.
+	 * @throws UnsupportedOperationException
+	 *             if the specified number of attributes is greater than 25
+	 */
+	SpoutCollector(final HashMap<String, Integer> numberOfAttributes,
+			final SourceContext<OUT> flinkContext) throws UnsupportedOperationException {
+		super(numberOfAttributes);
+		assert (flinkContext != null);
+		this.flinkContext = flinkContext;
+	}
+
+	@Override
+	protected List<Integer> doEmit(final OUT flinkTuple) {
+		this.flinkContext.collect(flinkTuple);
+		// TODO
+		return null;
+	}
+
+	@Override
+	public void reportError(final Throwable error) {
+		// not sure, if Flink can support this
+	}
+
+	@Override
+	public List<Integer> emit(final String streamId, final List<Object> tuple, final Object messageId) {
+		return this.tansformAndEmit(streamId, tuple);
+	}
+
+
+	@Override
+	public void emitDirect(final int taskId, final String streamId, final List<Object> tuple, final Object messageId) {
+		throw new UnsupportedOperationException("Direct emit is not supported by Flink");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
----------------------------------------------------------------------
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 0000000..914a19d
--- /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).<br />
+ * <br />
+ * Per default, {@link SpoutWrapper} calls the wrapped spout's {@link IRichSpout#nextTuple() nextTuple()} method in
+ * an infinite loop.<br />
+ * 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()}.<br />
+ * 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<OUT> extends RichParallelSourceFunction<OUT> {
+	private static final long serialVersionUID = -218340336648247605L;
+
+	/** Number of attributes of the spouts's output tuples per stream. */
+	private final HashMap<String, Integer> numberOfAttributes;
+	/** The wrapped {@link IRichSpout spout}. */
+	private final IRichSpout spout;
+	/** The wrapper of the given Flink collector. */
+	private SpoutCollector<OUT> 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<String>) 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<String>) 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<String> 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<String> 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<OUT> ctx) throws Exception {
+		this.collector = new SpoutCollector<OUT>(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}
+	 * <p/>
+	 * Sets the {@link #isRunning} flag to {@code false}.
+	 */
+	@Override
+	public void cancel() {
+		this.isRunning = false;
+	}
+
+	@Override
+	public void close() throws Exception {
+		this.spout.close();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/StormTuple.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/StormTuple.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/StormTuple.java
new file mode 100644
index 0000000..c9ab8e5
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/StormTuple.java
@@ -0,0 +1,327 @@
+/*
+ * 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;
+
+/*
+ * We do neither import
+ * 		backtype.storm.tuple.Tuple;
+ * nor
+ * 		org.apache.flink.api.java.tuple.Tuple
+ * to avoid confusion
+ */
+
+import backtype.storm.generated.GlobalStreamId;
+import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.MessageId;
+import backtype.storm.tuple.Values;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.List;
+
+/**
+ * {@link StormTuple} converts a Flink tuple of type {@code IN} into a Storm tuple.
+ */
+class StormTuple<IN> implements backtype.storm.tuple.Tuple {
+
+	/** The Storm representation of the original Flink tuple */
+	private final Values stormTuple;
+	/** The schema (ie, ordered field names) of the tuple */
+	private final Fields schema;
+
+	/**
+	 * Create a new Storm tuple from the given Flink tuple. The provided {@code nameIndexMap} is ignored for raw input
+	 * types.
+	 * 
+	 * @param flinkTuple
+	 * 		The Flink tuple to be converted.
+	 * @param schema
+	 * 		The schema (ie, ordered field names) of the tuple.
+	 */
+	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;
+
+			final int numberOfAttributes = t.getArity();
+			this.stormTuple = new Values();
+			for (int i = 0; i < numberOfAttributes; ++i) {
+				this.stormTuple.add(t.getField(i));
+			}
+		} else {
+			this.schema = null;
+			this.stormTuple = new Values(flinkTuple);
+		}
+	}
+
+	@Override
+	public int size() {
+		return this.stormTuple.size();
+	}
+
+	@Override
+	public boolean contains(final String field) {
+		if (this.schema != null) {
+			return this.schema.contains(field);
+		}
+
+		try {
+			this.getPublicMemberField(field);
+			return true;
+		} catch (NoSuchFieldException f) {
+			try {
+				this.getGetterMethod(field);
+				return true;
+			} catch (Exception g) {
+				return false;
+			}
+		} catch (Exception e) {
+			return false;
+		}
+	}
+
+	@Override
+	public Fields getFields() {
+		return this.schema;
+	}
+
+	@Override
+	public int fieldIndex(final String field) {
+		return this.schema.fieldIndex(field);
+	}
+
+	@Override
+	public List<Object> select(final Fields selector) {
+		return this.schema.select(selector, this.stormTuple);
+	}
+
+	@Override
+	public Object getValue(final int i) {
+		return this.stormTuple.get(i);
+	}
+
+	@Override
+	public String getString(final int i) {
+		return (String) this.stormTuple.get(i);
+	}
+
+	@Override
+	public Integer getInteger(final int i) {
+		return (Integer) this.stormTuple.get(i);
+	}
+
+	@Override
+	public Long getLong(final int i) {
+		return (Long) this.stormTuple.get(i);
+	}
+
+	@Override
+	public Boolean getBoolean(final int i) {
+		return (Boolean) this.stormTuple.get(i);
+	}
+
+	@Override
+	public Short getShort(final int i) {
+		return (Short) this.stormTuple.get(i);
+	}
+
+	@Override
+	public Byte getByte(final int i) {
+		return (Byte) this.stormTuple.get(i);
+	}
+
+	@Override
+	public Double getDouble(final int i) {
+		return (Double) this.stormTuple.get(i);
+	}
+
+	@Override
+	public Float getFloat(final int i) {
+		return (Float) this.stormTuple.get(i);
+	}
+
+	@Override
+	public byte[] getBinary(final int i) {
+		return (byte[]) this.stormTuple.get(i);
+	}
+
+	private Field getPublicMemberField(final String field) throws Exception {
+		assert (this.stormTuple.size() == 1);
+		return this.stormTuple.get(0).getClass().getField(field);
+	}
+
+	private Method getGetterMethod(final String field) throws Exception {
+		assert (this.stormTuple.size() == 1);
+		return this.stormTuple
+				.get(0)
+				.getClass()
+				.getMethod("get" + Character.toUpperCase(field.charAt(0)) + field.substring(1),
+						(Class<?>[]) null);
+	}
+
+	private Object getValueByPublicMember(final String field) throws Exception {
+		assert (this.stormTuple.size() == 1);
+		return getPublicMemberField(field).get(this.stormTuple.get(0));
+	}
+
+	private Object getValueByGetter(final String field) throws Exception {
+		assert (this.stormTuple.size() == 1);
+		return getGetterMethod(field).invoke(this.stormTuple.get(0), (Object[]) null);
+	}
+
+	@SuppressWarnings("unchecked")
+	public <T> T getValueByName(final String field) {
+		if (this.schema != null) {
+			return (T) this.getValue(this.schema.fieldIndex(field));
+		}
+		assert (this.stormTuple.size() == 1);
+
+		Exception e;
+		try {
+			// try public member
+			return (T) getValueByPublicMember(field);
+		} catch (NoSuchFieldException f) {
+			try {
+				// try getter-method
+				return (T) getValueByGetter(field);
+			} catch (Exception g) {
+				e = g;
+			}
+		} catch (Exception f) {
+			e = f;
+		}
+
+		throw new RuntimeException("Could not access field <" + field + ">", e);
+	}
+
+	@Override
+	public Object getValueByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public String getStringByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public Integer getIntegerByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public Long getLongByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public Boolean getBooleanByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public Short getShortByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public Byte getByteByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public Double getDoubleByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public Float getFloatByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public byte[] getBinaryByField(final String field) {
+		return getValueByName(field);
+	}
+
+	@Override
+	public List<Object> getValues() {
+		return this.stormTuple;
+	}
+
+	@Override
+	public GlobalStreamId getSourceGlobalStreamid() {
+		// not sure if Flink can support this
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public String getSourceComponent() {
+		// not sure if Flink can support this
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public int getSourceTask() {
+		// not sure if Flink can support this
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public String getSourceStreamId() {
+		// not sure if Flink can support this
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public MessageId getMessageId() {
+		// not sure if Flink can support this
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	public int hashCode() {
+		final int prime = 31;
+		int result = 1;
+		result = (prime * result) + ((this.stormTuple == null) ? 0 : this.stormTuple.hashCode());
+		return result;
+	}
+
+	@Override
+	public boolean equals(final Object obj) {
+		if (this == obj) {
+			return true;
+		}
+		if (obj == null) {
+			return false;
+		}
+		if (this.getClass() != obj.getClass()) {
+			return false;
+		}
+		final StormTuple<?> other = (StormTuple<?>) obj;
+		if (this.stormTuple == null) {
+			if (other.stormTuple != null) {
+				return false;
+			}
+		} else if (!this.stormTuple.equals(other.stormTuple)) {
+			return false;
+		}
+		return true;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
new file mode 100644
index 0000000..d529b6a
--- /dev/null
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
@@ -0,0 +1,266 @@
+/*
+ * 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.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.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 WrapperSetupHelper} is an helper class used by {@link AbstractStormSpoutWrapper} and
+ * {@link BoltWrapper}.
+ */
+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 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}.
+	 * 
+	 * @param spoutOrBolt
+	 *            The Storm {@link IRichSpout spout} or {@link IRichBolt bolt} to be used.
+	 * @param rawOutputs
+	 *            Contains stream names if a single attribute output stream, should not be of type
+	 *            {@link org.apache.flink.api.java.tuple.Tuple1 Tuple1} but be of a raw type. (Can be {@code null}.)
+	 * @return The number of attributes to be used for each stream.
+	 * @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].
+	 */
+	static HashMap<String, Integer> getNumberOfAttributes(final IComponent spoutOrBolt,
+			final Collection<String> rawOutputs)
+					throws IllegalArgumentException {
+		final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+		spoutOrBolt.declareOutputFields(declarer);
+
+		for (Entry<String, Integer> schema : declarer.outputSchemas.entrySet()) {
+			int declaredNumberOfAttributes = schema.getValue();
+			if ((declaredNumberOfAttributes < 0) || (declaredNumberOfAttributes > 25)) {
+				throw new IllegalArgumentException(
+						"Provided bolt declares non supported number of output attributes. Must be in range [0;25] but "
+								+ "was " + declaredNumberOfAttributes);
+			}
+
+			if (rawOutputs != null && rawOutputs.contains(schema.getKey())) {
+				if (declaredNumberOfAttributes != 1) {
+					throw new IllegalArgumentException(
+							"Ouput type is requested to be raw type, but provided bolt declares more then one output "
+									+ "attribute.");
+				}
+				schema.setValue(-1);
+			}
+		}
+
+		return declarer.outputSchemas;
+	}
+
+	/** 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" })
+	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<Integer, String> taskToComponents = new HashMap<Integer, String>();
+		final Map<String, List<Integer>> componentToSortedTasks = new HashMap<String, List<Integer>>();
+		final Map<String, Map<String, Fields>> componentToStreamToFields = new HashMap<String, Map<String, Fields>>();
+		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<Integer> workerTasks = new ArrayList<Integer>();
+		final Map<String, Object> defaultResources = new HashMap<String, Object>();
+		final Map<String, Object> userResources = new HashMap<String, Object>();
+		final Map<String, Object> executorData = new HashMap<String, Object>();
+		final Map registeredMetrics = new HashMap();
+		Atom openOrPrepareWasCalled = null;
+
+		if (stormTopology == null) {
+			// embedded mode
+			ComponentCommon common = new ComponentCommon();
+			common.set_parallelism_hint(dop);
+
+			HashMap<String, SpoutSpec> spouts = new HashMap<String, SpoutSpec>();
+			HashMap<String, Bolt> bolts = new HashMap<String, Bolt>();
+			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<String, StateSpoutSpec>());
+
+			taskId = context.getIndexOfThisSubtask();
+
+			List<Integer> sortedTasks = new ArrayList<Integer>(dop);
+			for (int i = 1; i <= dop; ++i) {
+				taskToComponents.put(i, operatorName);
+				sortedTasks.add(i);
+			}
+			componentToSortedTasks.put(operatorName, sortedTasks);
+
+			SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
+			spoutOrBolt.declareOutputFields(declarer);
+			componentToStreamToFields.put(operatorName, declarer.outputStreams);
+		} else {
+			// whole topology is built (ie, FlinkTopologyBuilder is used)
+			Map<String, SpoutSpec> spouts = stormTopology.get_spouts();
+			Map<String, Bolt> bolts = stormTopology.get_bolts();
+			Map<String, StateSpoutSpec> stateSpouts = stormTopology.get_state_spouts();
+
+			tid = 1;
+
+			for (Entry<String, SpoutSpec> 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<String, Bolt> 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<String, StateSpoutSpec> 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<Integer, String> taskToComponents,
+			final Map<String, List<Integer>> componentToSortedTasks,
+			final Map<String, Map<String, Fields>> componentToStreamToFields) {
+		final int parallelism_hint = common.get_parallelism_hint();
+		Integer taskId = null;
+
+		if (componentId.equals(operatorName)) {
+			taskId = tid + index;
+		}
+
+		List<Integer> sortedTasks = new ArrayList<Integer>(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<String, Fields> outputStreams = new HashMap<String, Fields>();
+		for(Entry<String, StreamInfo> outStream : common.get_streams().entrySet()) {
+			outputStreams.put(outStream.getKey(), new Fields(outStream.getValue().get_output_fields()));
+		}
+		componentToStreamToFields.put(componentId, outputStreams);
+
+		return taskId;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java
new file mode 100644
index 0000000..49de476
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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.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.storm.api.FlinkOutputFieldsDeclarer;
+import org.apache.flink.storm.util.AbstractTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.LinkedList;
+
+public class FlinkOutputFieldsDeclarerTest extends AbstractTest {
+
+
+
+	@Test
+	public void testNull() {
+		Assert.assertNull(new FlinkOutputFieldsDeclarer().getOutputType(null));
+	}
+
+	@Test
+	public void testDeclare() {
+		for (int i = 0; i < 2; ++i) { // test case: simple / non-direct
+			for (int j = 1; j < 2; ++j) { // number of streams
+				for (int k = 0; k <= 25; ++k) { // number of attributes
+					this.runDeclareTest(i, j, k);
+				}
+			}
+		}
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareSimpleToManyAttributes() {
+		this.runDeclareTest(0, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareNonDirectToManyAttributes() {
+		this.runDeclareTest(1, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareDefaultStreamToManyAttributes() {
+		this.runDeclareTest(2, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testDeclareFullToManyAttributes() {
+		this.runDeclareTest(3, this.r.nextBoolean() ? 1 : 2, 26);
+	}
+
+	private void runDeclareTest(final int testCase, final int numberOfStreams,
+			final int numberOfAttributes) {
+		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+
+		String[] streams = null;
+		if (numberOfStreams > 1 || r.nextBoolean()) {
+			streams = new String[numberOfStreams];
+			for (int i = 0; i < numberOfStreams; ++i) {
+				streams[i] = "stream" + i;
+			}
+		}
+
+		final String[] attributes = new String[numberOfAttributes];
+		for (int i = 0; i < attributes.length; ++i) {
+			attributes[i] = "a" + i;
+		}
+
+		switch (testCase) {
+		case 0:
+			this.declareSimple(declarer, streams, attributes);
+			break;
+		default:
+			this.declareNonDirect(declarer, streams, attributes);
+		}
+
+		if (streams == null) {
+			streams = new String[] { Utils.DEFAULT_STREAM_ID };
+		}
+
+		for (String stream : streams) {
+			final TypeInformation<?> type = declarer.getOutputType(stream);
+
+			if (numberOfAttributes == 1) {
+				Assert.assertEquals(type.getClass(), GenericTypeInfo.class);
+				Assert.assertEquals(type.getTypeClass(), Object.class);
+			} else {
+				Assert.assertEquals(numberOfAttributes, type.getArity());
+				Assert.assertTrue(type.isTupleType());
+			}
+		}
+	}
+
+	private void declareSimple(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
+			final String[] attributes) {
+
+		if (streams != null) {
+			for (String stream : streams) {
+				declarer.declareStream(stream, new Fields(attributes));
+			}
+		} else {
+			declarer.declare(new Fields(attributes));
+		}
+	}
+
+	private void declareNonDirect(final FlinkOutputFieldsDeclarer declarer, final String[] streams,
+			final String[] attributes) {
+
+		if (streams != null) {
+			for (String stream : streams) {
+				declarer.declareStream(stream, false, new Fields(attributes));
+			}
+		} else {
+			declarer.declare(false, new Fields(attributes));
+		}
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testUndeclared() {
+		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+		declarer.getOutputType("unknownStreamId");
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareDirect() {
+		new FlinkOutputFieldsDeclarer().declare(true, null);
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testDeclareDirect2() {
+		new FlinkOutputFieldsDeclarer().declareStream(Utils.DEFAULT_STREAM_ID, true, null);
+	}
+
+	@Test
+	public void testGetGroupingFieldIndexes() {
+		final int numberOfAttributes = 5 + this.r.nextInt(21);
+		final String[] attributes = new String[numberOfAttributes];
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			attributes[i] = "a" + i;
+		}
+
+		final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer();
+		declarer.declare(new Fields(attributes));
+
+		final int numberOfKeys = 1 + this.r.nextInt(25);
+		final LinkedList<String> groupingFields = new LinkedList<String>();
+		final boolean[] indexes = new boolean[numberOfAttributes];
+
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			if (this.r.nextInt(26) < numberOfKeys) {
+				groupingFields.add(attributes[i]);
+				indexes[i] = true;
+			} else {
+				indexes[i] = false;
+			}
+		}
+
+		final int[] expectedResult = new int[groupingFields.size()];
+		int j = 0;
+		for (int i = 0; i < numberOfAttributes; ++i) {
+			if (indexes[i]) {
+				expectedResult[j++] = i;
+			}
+		}
+
+		final int[] result = declarer.getGroupingFieldIndexes(Utils.DEFAULT_STREAM_ID,
+				groupingFields);
+
+		Assert.assertEquals(expectedResult.length, result.length);
+		for (int i = 0; i < expectedResult.length; ++i) {
+			Assert.assertEquals(expectedResult[i], result[i]);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
new file mode 100644
index 0000000..e6fb8e5
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.api;
+
+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;
+
+public class FlinkTopologyBuilderTest {
+
+	@Test(expected = RuntimeException.class)
+	public void testUnknowSpout() {
+		FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
+		builder.setSpout("spout", new TestSpout());
+		builder.setBolt("bolt", new TestBolt()).shuffleGrouping("unknown");
+		builder.createTopology();
+	}
+
+	@Test(expected = RuntimeException.class)
+	public void testUnknowBolt() {
+		FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
+		builder.setSpout("spout", new TestSpout());
+		builder.setBolt("bolt1", new TestBolt()).shuffleGrouping("spout");
+		builder.setBolt("bolt2", new TestBolt()).shuffleGrouping("unknown");
+		builder.createTopology();
+	}
+
+	@Test(expected = RuntimeException.class)
+	public void testUndeclaredStream() {
+		FlinkTopologyBuilder builder = new FlinkTopologyBuilder();
+		builder.setSpout("spout", new TestSpout());
+		builder.setBolt("bolt", new TestBolt()).shuffleGrouping("spout");
+		builder.createTopology();
+	}
+
+	@Test
+	public void testFieldsGroupingOnMultipleSpoutOutputStreams() {
+		FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
+
+		flinkBuilder.setSpout("spout", new TestDummySpout());
+		flinkBuilder.setBolt("sink", new TestSink()).fieldsGrouping("spout",
+				TestDummySpout.spoutStreamId, new Fields("id"));
+
+		flinkBuilder.createTopology();
+	}
+
+	@Test
+	public void testFieldsGroupingOnMultipleBoltOutputStreams() {
+		FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
+
+		flinkBuilder.setSpout("spout", new TestDummySpout());
+		flinkBuilder.setBolt("bolt", new TestDummyBolt()).shuffleGrouping("spout");
+		flinkBuilder.setBolt("sink", new TestSink()).fieldsGrouping("bolt",
+				TestDummyBolt.groupingStreamId, new Fields("id"));
+
+		flinkBuilder.createTopology();
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java
new file mode 100644
index 0000000..9d04ca5
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.api;
+
+import org.apache.flink.storm.api.FlinkTopology;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class FlinkTopologyTest {
+
+	@Test
+	public void testDefaultParallelism() {
+		final FlinkTopology topology = new FlinkTopology();
+		Assert.assertEquals(1, topology.getParallelism());
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testExecute() throws Exception {
+		new FlinkTopology().execute();
+	}
+
+	@Test(expected = UnsupportedOperationException.class)
+	public void testExecuteWithName() throws Exception {
+		new FlinkTopology().execute(null);
+	}
+
+	@Test
+	public void testNumberOfTasks() {
+		final FlinkTopology topology = new FlinkTopology();
+
+		Assert.assertEquals(0, topology.getNumberOfTasks());
+
+		topology.increaseNumberOfTasks(3);
+		Assert.assertEquals(3, topology.getNumberOfTasks());
+
+		topology.increaseNumberOfTasks(2);
+		Assert.assertEquals(5, topology.getNumberOfTasks());
+
+		topology.increaseNumberOfTasks(8);
+		Assert.assertEquals(13, topology.getNumberOfTasks());
+	}
+
+	@Test(expected = AssertionError.class)
+	public void testAssert() {
+		new FlinkTopology().increaseNumberOfTasks(0);
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java
new file mode 100644
index 0000000..74ea67e
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java
@@ -0,0 +1,48 @@
+/*
+ * 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.api;
+
+import java.util.Map;
+
+import backtype.storm.task.OutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichBolt;
+import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.Tuple;
+
+public class TestBolt implements IRichBolt {
+	private static final long serialVersionUID = -667148827441397683L;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) {}
+
+	@Override
+	public void execute(Tuple input) {}
+
+	@Override
+	public void cleanup() {}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java
new file mode 100644
index 0000000..4abb604
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java
@@ -0,0 +1,59 @@
+/*
+ * 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.api;
+
+import java.util.Map;
+
+import backtype.storm.spout.SpoutOutputCollector;
+import backtype.storm.task.TopologyContext;
+import backtype.storm.topology.IRichSpout;
+import backtype.storm.topology.OutputFieldsDeclarer;
+
+public class TestSpout implements IRichSpout {
+	private static final long serialVersionUID = -4884029383198924007L;
+
+	@SuppressWarnings("rawtypes")
+	@Override
+	public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {}
+
+	@Override
+	public void close() {}
+
+	@Override
+	public void activate() {}
+
+	@Override
+	public void deactivate() {}
+
+	@Override
+	public void nextTuple() {}
+
+	@Override
+	public void ack(Object msgId) {}
+
+	@Override
+	public void fail(Object msgId) {}
+
+	@Override
+	public void declareOutputFields(OutputFieldsDeclarer declarer) {}
+
+	@Override
+	public Map<String, Object> getComponentConfiguration() {
+		return null;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/4cb96708/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestTopologyBuilder.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestTopologyBuilder.java
new file mode 100644
index 0000000..7bea94c
--- /dev/null
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestTopologyBuilder.java
@@ -0,0 +1,29 @@
+/*
+ * 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.api;
+
+import org.apache.flink.storm.api.FlinkTopologyBuilder;
+
+import backtype.storm.generated.StormTopology;
+
+public class TestTopologyBuilder extends FlinkTopologyBuilder {
+	@Override
+	public StormTopology getStormTopology() {
+		return super.getStormTopology();
+	}
+}