Skip to content

Commit

Permalink
[storm-compat] Added Storm compatibility word count examples
Browse files Browse the repository at this point in the history
  • Loading branch information
mjsax authored and mbalassi committed Jun 14, 2015
1 parent 56e013f commit 000b5d5
Show file tree
Hide file tree
Showing 6 changed files with 665 additions and 0 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.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;

import backtype.storm.topology.IRichBolt;





/**
* 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>
* The input is a plain text file with lines separated by newline characters.
*
* <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>
* This example shows how to:
* <ul>
* <li>use a Storm bolt within a Flink Streaming program.
* </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>(new String(), new 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"
.groupBy(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: 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;
}

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);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.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.StormFileSpout;
import org.apache.flink.stormcompatibility.wordcount.stormoperators.StormInMemorySpout;
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;

import backtype.storm.topology.IRichSpout;





/**
* 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>
* The input is a plain text file with lines separated by newline characters.
*
* <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>
* This example shows how to:
* <ul>
* <li>use a Storm bolt within a Flink Streaming program.
* </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"
.groupBy(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, new Integer(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: 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;
}

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];
final DataStream<String> stream = env.addSource(
new StormFiniteSpoutWrapper<String>(new StormFileSpout(localFile), true),
TypeExtractor.getForClass(String.class)).setParallelism(1);
return stream;
}

return env.addSource(new StormFiniteSpoutWrapper<String>(new StormInMemorySpout(), true),
TypeExtractor.getForClass(String.class));

}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT 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.examples.java.wordcount.util.WordCountData;
import org.apache.flink.stormcompatibility.api.FlinkLocalCluster;
import org.apache.flink.stormcompatibility.api.FlinkTopologyBuilder;

import backtype.storm.LocalCluster;
import backtype.storm.generated.StormTopology;
import backtype.storm.utils.Utils;





/**
* 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}.
*
* 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>
* The input is a plain text file with lines separated by newline characters.
*
* <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>
* This example shows how to:
* <ul>
* <li>run a regular Storm program locally on Flink
* </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(5 * 1000);

// TODO kill does no do anything so far
cluster.killTopology(topologyId);
cluster.shutdown();
}

}
Loading

0 comments on commit 000b5d5

Please sign in to comment.