You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mb...@apache.org on 2015/06/15 11:32:57 UTC

[07/27] flink git commit: [storm-compat] Added ITCases to Storm compatibility examples

[storm-compat] Added ITCases to Storm compatibility examples


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

Branch: refs/heads/master
Commit: fa13e49bb3e56dacd0d84a0a47949d33abe91a37
Parents: 000b5d5
Author: mjsax <mj...@informatik.hu-berlin.de>
Authored: Thu May 14 12:59:38 2015 +0200
Committer: mbalassi <mb...@apache.org>
Committed: Sun Jun 14 22:59:36 2015 +0200

----------------------------------------------------------------------
 .../api/FlinkTestCluster.java                   | 120 +++++++++++++++++++
 .../wordcount/BoltTokenizerWordCountITCase.java |  49 ++++++++
 .../wordcount/SpoutSourceWordCountITCase.java   |  49 ++++++++
 .../wordcount/StormWordCountLocalITCase.java    |  52 ++++++++
 4 files changed, 270 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fa13e49b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.java b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.java
new file mode 100644
index 0000000..2d64576
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/api/FlinkTestCluster.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.TestStreamEnvironment;
+
+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;
+
+
+
+
+
+/**
+ * {@link FlinkTestCluster} mimics a Storm {@link LocalCluster} for ITCases via a {@link TestStreamEnvironment}.
+ */
+public class FlinkTestCluster extends FlinkLocalCluster {
+	
+	@Override
+	public void submitTopology(final String topologyName, final Map<?, ?> conf, final FlinkTopology topology)
+		throws Exception {
+		this.submitTopologyWithOpts(topologyName, conf, topology, null);
+	}
+	
+	@Override
+	public void submitTopologyWithOpts(final String topologyName, final Map<?, ?> conf, final FlinkTopology topology, final SubmitOptions submitOpts)
+		throws Exception {
+		final TestStreamEnvironment env = (TestStreamEnvironment)StreamExecutionEnvironment.getExecutionEnvironment();
+		env.start(topology.getStreamGraph().getJobGraph(topologyName));
+	}
+	
+	@Override
+	public void killTopology(final String topologyName) {
+		// TODO Auto-generated method stub
+	}
+	
+	@Override
+	public void killTopologyWithOpts(final String name, final KillOptions options) {
+		// TODO Auto-generated method stub
+	}
+	
+	@Override
+	public void activate(final String topologyName) {
+		// TODO Auto-generated method stub
+	}
+	
+	@Override
+	public void deactivate(final String topologyName) {
+		// TODO Auto-generated method stub
+	}
+	
+	@Override
+	public void rebalance(final String name, final RebalanceOptions options) {
+		// TODO Auto-generated method stub
+	}
+	
+	@Override
+	public void shutdown() {
+		final TestStreamEnvironment env = (TestStreamEnvironment)StreamExecutionEnvironment.getExecutionEnvironment();
+		try {
+			env.shutdown();
+		} catch(final InterruptedException e) {
+			e.printStackTrace();
+		}
+	}
+	
+	@Override
+	public String getTopologyConf(final String id) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+	
+	@Override
+	public StormTopology getTopology(final String id) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+	
+	@Override
+	public ClusterSummary getClusterInfo() {
+		// TODO Auto-generated method stub
+		return null;
+	}
+	
+	@Override
+	public TopologyInfo getTopologyInfo(final String id) {
+		// TODO Auto-generated method stub
+		return null;
+	}
+	
+	@Override
+	public Map<?, ?> getState() {
+		// TODO Auto-generated method stub
+		return null;
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fa13e49b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.java
new file mode 100644
index 0000000..82fc119
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/BoltTokenizerWordCountITCase.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.stormcompatibility.wordcount;
+
+import org.apache.flink.stormcompatibility.wordcount.BoltTokenizerWordCount;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.test.testdata.WordCountData;
+
+
+
+
+
+public class BoltTokenizerWordCountITCase extends StreamingProgramTestBase {
+	
+	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 {
+		this.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/fa13e49b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.java
new file mode 100644
index 0000000..5f80684
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/SpoutSourceWordCountITCase.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.stormcompatibility.wordcount;
+
+import org.apache.flink.stormcompatibility.wordcount.SpoutSourceWordCount;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.test.testdata.WordCountData;
+
+
+
+
+
+public class SpoutSourceWordCountITCase extends StreamingProgramTestBase {
+	
+	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 {
+		this.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/fa13e49b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.java
new file mode 100644
index 0000000..237c598
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-storm-examples/src/test/java/org/apache/flink/stormcompatibility/wordcount/StormWordCountLocalITCase.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.stormcompatibility.wordcount;
+
+import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
+import org.apache.flink.stormcompatibility.api.FlinkTestCluster;
+import org.apache.flink.stormcompatibility.wordcount.StormWordCountLocal;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.apache.flink.test.testdata.WordCountData;
+
+
+
+
+
+public class StormWordCountLocalITCase extends StreamingProgramTestBase {
+	
+	protected String textPath;
+	protected String resultPath;
+	
+	@Override
+	protected void preSubmit() throws Exception {
+		FlinkLocalCluster.initialize(new FlinkTestCluster());
+		this.textPath = this.createTempFile("text.txt", WordCountData.TEXT);
+		this.resultPath = this.getTempDirPath("result");
+	}
+	
+	@Override
+	protected void postSubmit() throws Exception {
+		this.compareResultsByLinesInMemory(WordCountData.STREAMING_COUNTS_AS_TUPLES, this.resultPath);
+	}
+	
+	@Override
+	protected void testProgram() throws Exception {
+		StormWordCountLocal.main(new String[] {this.textPath, this.resultPath});
+	}
+	
+}