├── project
├── build.properties
└── plugins.sbt
├── version.sbt
├── .gitignore
├── .travis.yml
├── src
├── test
│ ├── resources
│ │ └── logback-test.xml
│ └── scala
│ │ └── storm
│ │ └── scala
│ │ ├── examples
│ │ ├── ExclamationTopology.scala
│ │ ├── WordCountTopology.scala
│ │ └── AggregationTopology.scala
│ │ └── test
│ │ └── MetricsTopologyTest.scala
└── main
│ └── scala
│ └── storm
│ └── scala
│ └── dsl
│ ├── ShutdownFunc.scala
│ ├── SetupFunc.scala
│ ├── FunctionalTrident.scala
│ ├── StormBolt.scala
│ ├── StormSpout.scala
│ ├── StormTuple.scala
│ └── StormMetricsConsumer.scala
├── LICENSE.md
├── publish.sbt
├── defaults.yaml
└── README.md
/project/build.properties:
--------------------------------------------------------------------------------
1 | sbt.version=0.13.5
2 |
--------------------------------------------------------------------------------
/version.sbt:
--------------------------------------------------------------------------------
1 |
2 | version in ThisBuild := "0.3.0-SNAPSHOT"
3 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | .idea
2 | .idea_modules
3 | *.iml
4 | target
5 | project/boot
6 | project/target
7 |
--------------------------------------------------------------------------------
/project/plugins.sbt:
--------------------------------------------------------------------------------
1 | addSbtPlugin("com.typesafe.sbt" % "sbt-pgp" % "0.8.2")
2 |
3 | addSbtPlugin("com.github.gseitz" % "sbt-release" % "0.8.3")
4 |
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | language: scala
2 | scala:
3 | - 2.10.4
4 | jdk:
5 | - oraclejdk7
6 | - openjdk6
7 |
8 | script: sbt ++$TRAVIS_SCALA_VERSION test
9 |
--------------------------------------------------------------------------------
/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 | %d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 |
18 |
--------------------------------------------------------------------------------
/LICENSE.md:
--------------------------------------------------------------------------------
1 | Copyright (c) 2011 Evan Chan
2 |
3 | Licensed under the Apache License, Version 2.0 (the "License");
4 | you may not use this file except in compliance with the License.
5 | You may obtain a copy of the License at
6 |
7 | http://www.apache.org/licenses/LICENSE-2.0
8 |
9 | Unless required by applicable law or agreed to in writing, software
10 | distributed under the License is distributed on an "AS IS" BASIS,
11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | See the License for the specific language governing permissions and
13 | limitations under the License.
--------------------------------------------------------------------------------
/src/main/scala/storm/scala/dsl/ShutdownFunc.scala:
--------------------------------------------------------------------------------
1 | package storm.scala.dsl
2 |
3 | // @author rwagner
4 | //
5 | // A trait with a DSL for defining shutdown code, that should happen when an instance
6 | // of a Bolt shuts down up. Acts as a counterpart for SetupFunc.
7 | //
8 | // Use it like this:
9 | // class MyBolt extends StormBolt(List("word")) {
10 | // val myResouce= AnyResource(config)
11 | // setup { myResouce.open() }
12 | // shutdown { myResouce.close() }
13 | // }
14 | trait ShutdownFunc {
15 |
16 | // register a shutdown function
17 | def shutdown(sf: => Unit) = _shutdownFunctions ::= sf _
18 |
19 | // fire all registered shutdown functions
20 | protected def _cleanup() = _shutdownFunctions.foreach(_())
21 |
22 | // list of registered shutdown functions
23 | private var _shutdownFunctions: List[() => Unit] = Nil
24 | }
--------------------------------------------------------------------------------
/src/main/scala/storm/scala/dsl/SetupFunc.scala:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2011 Evan Chan
2 |
3 | package storm.scala.dsl
4 |
5 | // A trait with a DSL for defining initialization that should happen when every instance
6 | // of a Bolt starts up. Initialization logic in general should not go in the class
7 | // constructor, because that will run only on the machine where you submit a topology,
8 | // not on the nodes themselves. Only prepare() (and thus setup) get run on each node.
9 | //
10 | // Use it like this:
11 | // class MyBolt extends StormBolt(List("word")) {
12 | // var myIterator: Iterator[Int] = _
13 | // setup { myIterator = ... }
14 | // }
15 | trait SetupFunc {
16 |
17 | // fire all setup functions
18 | def _setup() = setupFuncs.foreach(_())
19 |
20 | // register a setup function
21 | def setup(func: => Unit) = { setupFuncs ::= func _ }
22 |
23 | // registered setup functions
24 | private var setupFuncs: List[() => Unit] = Nil
25 | }
--------------------------------------------------------------------------------
/publish.sbt:
--------------------------------------------------------------------------------
1 | publishMavenStyle := true
2 |
3 | publishTo <<= version { (v: String) =>
4 | if (v.trim.endsWith("SNAPSHOT")) Some(Opts.resolver.sonatypeSnapshots) else Some(Opts.resolver.sonatypeStaging)
5 | }
6 |
7 | publishArtifact in Test := false
8 |
9 | pomIncludeRepository := { _ => false }
10 |
11 | pomExtra := (
12 | https://github.com/velvia/ScalaStorm
13 |
14 |
15 | Apache
16 | http://www.apache.org/licenses/LICENSE-2.0.html
17 | repo
18 |
19 |
20 |
21 | git@github.com:velvia/ScalaStorm.git
22 | scm:git:git@github.com:velvia/ScalaStorm.git
23 |
24 |
25 |
26 | velvia
27 | Evan Chan
28 | http://github.com/velvia
29 |
30 | )
31 |
32 | releaseSettings
33 |
--------------------------------------------------------------------------------
/src/test/scala/storm/scala/examples/ExclamationTopology.scala:
--------------------------------------------------------------------------------
1 | package storm.scala.examples
2 |
3 | import storm.scala.dsl._
4 | import org.apache.storm.Config
5 | import org.apache.storm.LocalCluster
6 | import org.apache.storm.testing.TestWordSpout
7 | import org.apache.storm.topology.TopologyBuilder
8 | import org.apache.storm.tuple.{Fields, Tuple, Values}
9 | import scala.language.postfixOps
10 |
11 | class ExclamationBolt extends StormBolt(outputFields = List("word")) {
12 | def execute(t: Tuple) {
13 | t emit(t.getString(0) + "!!!")
14 | t ack
15 | }
16 | }
17 |
18 | object ExclamationTopology {
19 | def main(args: Array[String]) = {
20 | val builder = new TopologyBuilder()
21 |
22 | builder.setSpout("words", new TestWordSpout(), 10)
23 | builder.setBolt("exclaim1", new ExclamationBolt, 3)
24 | .shuffleGrouping("words")
25 | builder.setBolt("exclaim2", new ExclamationBolt, 2)
26 | .shuffleGrouping("exclaim1")
27 |
28 | val conf = new Config()
29 | conf setDebug true
30 |
31 | val cluster = new LocalCluster()
32 | cluster.submitTopology("test", conf, builder.createTopology())
33 | Thread sleep 10000
34 | cluster.killTopology("test")
35 | cluster.shutdown()
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/src/main/scala/storm/scala/dsl/FunctionalTrident.scala:
--------------------------------------------------------------------------------
1 | package storm.scala.dsl
2 |
3 | import collection.JavaConversions._
4 |
5 | import org.apache.storm.trident.tuple.TridentTuple
6 | import org.apache.storm.trident.operation.{TridentCollector, BaseFunction}
7 | import org.apache.storm.tuple.Fields
8 | import scala.language.implicitConversions
9 |
10 | /**
11 | * Functional DSL for Trident so you can easily use Scala closures with Trident.
12 | */
13 | object FunctionalTrident {
14 | class MapFuncT1(func: TridentTuple => Any) extends BaseFunction {
15 | def execute(tuple: TridentTuple, collector: TridentCollector) {
16 | collector.emit(List(func(tuple).asInstanceOf[AnyRef]))
17 | }
18 | }
19 |
20 | class FlatMapFuncT1(func: TridentTuple => Seq[Any]) extends BaseFunction {
21 | def execute(tuple: TridentTuple, collector: TridentCollector) {
22 | func(tuple).foreach { thing => collector.emit(List(thing.asInstanceOf[AnyRef])) }
23 | }
24 | }
25 |
26 | class FunctionalStream(origStream: org.apache.storm.trident.Stream) {
27 | // Example usage:
28 | // stream.map("sentence" -> "numwords") { _.getString(0).split(" ").length }
29 | def map(fieldMapping: (String, String))(mapFunc: TridentTuple => Any) =
30 | origStream.each(new Fields(fieldMapping._1),
31 | new MapFuncT1(mapFunc),
32 | new Fields(fieldMapping._2))
33 |
34 | // Example usage:
35 | // stream.flatMap("sentence" -> "words") { _.getString(0).split(" ") }
36 | def flatMap(fieldMapping: (String, String))(mapFunc: TridentTuple => Seq[Any]) =
37 | origStream.each(new Fields(fieldMapping._1),
38 | new FlatMapFuncT1(mapFunc),
39 | new Fields(fieldMapping._2))
40 | }
41 |
42 | implicit def TridentStreamToFunctionalStream(stream: org.apache.storm.trident.Stream) =
43 | new FunctionalStream(stream)
44 | }
45 |
--------------------------------------------------------------------------------
/src/test/scala/storm/scala/examples/WordCountTopology.scala:
--------------------------------------------------------------------------------
1 | package storm.scala.examples
2 |
3 | import storm.scala.dsl._
4 | import org.apache.storm.Config
5 | import org.apache.storm.LocalCluster
6 | import org.apache.storm.topology.TopologyBuilder
7 | import org.apache.storm.tuple.{Fields, Tuple, Values}
8 | import collection.mutable.{Map, HashMap}
9 | import util.Random
10 | import scala.language.postfixOps
11 |
12 |
13 | class RandomSentenceSpout extends StormSpout(outputFields = List("sentence")) {
14 | val sentences = List("the cow jumped over the moon",
15 | "an apple a day keeps the doctor away",
16 | "four score and seven years ago",
17 | "snow white and the seven dwarfs",
18 | "i am at two with nature")
19 | def nextTuple = {
20 | Thread sleep 100
21 | emit (sentences(Random.nextInt(sentences.length)))
22 | }
23 | }
24 |
25 |
26 | // An example of using matchSeq for Scala pattern matching of Storm tuples
27 | // plus using the emit and ack DSLs.
28 | class SplitSentence extends StormBolt(outputFields = List("word")) {
29 | def execute(t: Tuple) = t matchSeq {
30 | case Seq(sentence: String) => sentence split " " foreach
31 | { word => using anchor t emit (word) }
32 | t ack
33 | }
34 | }
35 |
36 |
37 | class WordCount extends StormBolt(List("word", "count")) {
38 | var counts: Map[String, Int] = _
39 | setup {
40 | counts = new HashMap[String, Int]().withDefaultValue(0)
41 | }
42 | def execute(t: Tuple) = t matchSeq {
43 | case Seq(word: String) =>
44 | counts(word) += 1
45 | using anchor t emit (word, counts(word))
46 | t ack
47 | }
48 | }
49 |
50 |
51 | object WordCountTopology {
52 | def main(args: Array[String]) = {
53 | val builder = new TopologyBuilder
54 |
55 | builder.setSpout("randsentence", new RandomSentenceSpout, 5)
56 | builder.setBolt("split", new SplitSentence, 8)
57 | .shuffleGrouping("randsentence")
58 | builder.setBolt("count", new WordCount, 12)
59 | .fieldsGrouping("split", new Fields("word"))
60 |
61 | val conf = new Config
62 | conf.setDebug(true)
63 | conf.setMaxTaskParallelism(3)
64 |
65 | val cluster = new LocalCluster
66 | cluster.submitTopology("word-count", conf, builder.createTopology)
67 | Thread sleep 10000
68 | cluster.shutdown
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/defaults.yaml:
--------------------------------------------------------------------------------
1 | ########### These all have default values as shown
2 | ########### Additional configuration goes into storm.yaml
3 |
4 | java.library.path: "/usr/local/lib:/opt/local/lib:/usr/lib"
5 |
6 | ### storm.* configs are general configurations
7 | # the local dir is where jars are kept
8 | storm.local.dir: "/mnt/storm"
9 | storm.zookeeper.port: 2181
10 | storm.zookeeper.root: "/storm"
11 | storm.zookeeper.session.timeout: 20000
12 | storm.cluster.mode: "distributed" # can be distributed or local
13 | storm.local.mode.zmq: false
14 |
15 | ### nimbus.* configs are for the master
16 | nimbus.thrift.port: 6627
17 | nimbus.childopts: "-Xmx1024m"
18 | nimbus.task.timeout.secs: 30
19 | nimbus.supervisor.timeout.secs: 60
20 | nimbus.monitor.freq.secs: 10
21 | nimbus.task.launch.secs: 240
22 | nimbus.reassign: true
23 | nimbus.file.copy.expiration.secs: 600
24 |
25 | ### supervisor.* configs are for node supervisors
26 | # Define the amount of workers that can be run on this machine. Each worker is assigned a port to use for communication
27 | supervisor.slots.ports:
28 | - 6700
29 | - 6701
30 | - 6702
31 | - 6703
32 | supervisor.childopts: "-Xmx1024m"
33 | #how long supervisor will wait to ensure that a worker process is started
34 | supervisor.worker.start.timeout.secs: 240
35 | #how long between heartbeats until supervisor considers that worker dead and tries to restart it
36 | supervisor.worker.timeout.secs: 30
37 | #how frequently the supervisor checks on the status of the processes it's monitoring and restarts if necessary
38 | supervisor.monitor.frequency.secs: 3
39 | #how frequently the supervisor heartbeats to the cluster state (for nimbus)
40 | supervisor.heartbeat.frequency.secs: 5
41 | supervisor.enable: true
42 |
43 | ### worker.* configs are for task workers
44 | worker.childopts: "-Xmx768m"
45 | worker.heartbeat.frequency.secs: 1
46 |
47 | task.heartbeat.frequency.secs: 3
48 | task.refresh.poll.secs: 10
49 |
50 | zmq.threads: 1
51 | zmq.linger.millis: 5000
52 |
53 | ### topology.* configs are for specific executing storms
54 | topology.debug: false
55 | topology.optimize: true
56 | topology.workers: 1
57 | topology.ackers: 1
58 | # maximum amount of time a message has to complete before it's considered failed
59 | topology.message.timeout.secs: 30
60 | topology.skip.missing.serializations: false
61 | topology.max.task.parallelism: null
62 | topology.max.spout.pending: null
63 | topology.state.synchronization.timeout.secs: 60
64 | topology.stats.sample.rate: 0.05
65 |
66 |
--------------------------------------------------------------------------------
/src/main/scala/storm/scala/dsl/StormBolt.scala:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2011 Evan Chan
2 |
3 | package storm.scala.dsl
4 |
5 | import org.apache.storm.topology.base.BaseRichBolt
6 | import org.apache.storm.topology.OutputFieldsDeclarer
7 | import org.apache.storm.tuple.{Fields, Tuple}
8 | import org.apache.storm.task.OutputCollector
9 | import org.apache.storm.task.TopologyContext
10 | import scala.language.implicitConversions
11 | import java.util.{Map => JMap}
12 |
13 | // The StormBolt class is an implementation of IRichBolt which
14 | // provides a Scala DSL for making Bolt development concise.
15 | // To use, extend this class and implement the execute(t: Tuple) method.
16 | //
17 | // The following DSLs for emitting are supported:
18 | // using anchor emit (...)
19 | // anchor() emit (...)
20 | // emit (...)
21 | // using no anchor emit (...)
22 | abstract class StormBolt(val streamToFields: collection.Map[String, List[String]])
23 | extends BaseRichBolt with SetupFunc with ShutdownFunc with BoltDsl {
24 | var _context: TopologyContext = _
25 | var _conf: JMap[_, _] = _
26 |
27 | // A constructor for the common case when you just want to output to the default stream
28 | def this(outputFields: List[String]) = { this(Map("default" -> outputFields)) }
29 |
30 | def prepare(conf:JMap[_, _], context:TopologyContext, collector:OutputCollector) {
31 | _collector = collector
32 | _context = context
33 | _conf = conf
34 | _setup()
35 | }
36 |
37 | def declareOutputFields(declarer: OutputFieldsDeclarer) {
38 | streamToFields foreach { case(stream, fields) =>
39 | declarer.declareStream(stream, new Fields(fields:_*))
40 | }
41 | }
42 |
43 | override def cleanup() = _cleanup()
44 | }
45 |
46 | /**
47 | * This trait allows the bolt emit DSL to be used outside of the main bolt class, say in an Actor or
48 | * separate thread. To use it you just need to initialise _collector before using it.
49 | */
50 | trait BoltDsl {
51 | var _collector: OutputCollector = _
52 |
53 | // Declare an anchor for emitting a tuple
54 | def anchor(tuple: Tuple) = new StormTuple(_collector, tuple)
55 |
56 | def anchor(tuples: List[Tuple]) = new StormTupleList(_collector, tuples)
57 |
58 | // Use this for unanchored emits:
59 | // using no anchor emit (val1, val2, ...)
60 | def no(s: String) = new UnanchoredEmit(_collector)
61 |
62 | val anchor = ""
63 |
64 | // Combine with anchor for a cool DSL like this:
65 | // using anchor t emit (val1, val2, ..)
66 | def using = this
67 |
68 | // implicitly convert to a stormTuple for easy emit syntax like
69 | // tuple emit (val1, val2, ...)
70 | implicit def stormTupleConvert(tuple: Tuple) =
71 | new StormTuple(_collector, tuple)
72 |
73 | implicit def stormTupleListConverter(tuples: Seq[Tuple]) =
74 | new StormTupleList(_collector, tuples)
75 | }
76 |
--------------------------------------------------------------------------------
/src/main/scala/storm/scala/dsl/StormSpout.scala:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2011 Evan Chan
2 |
3 | package storm.scala.dsl
4 |
5 | import org.apache.storm.task.TopologyContext
6 | import org.apache.storm.spout.SpoutOutputCollector
7 | import org.apache.storm.topology.OutputFieldsDeclarer
8 | import org.apache.storm.topology.base.BaseRichSpout
9 | import org.apache.storm.tuple.Fields
10 | import collection.JavaConverters._
11 | import collection.JavaConversions._
12 | import java.util.{Map => JMap}
13 |
14 | abstract class StormSpout(val streamToFields: collection.Map[String, List[String]],
15 | val isDistributed: Boolean) extends BaseRichSpout with SetupFunc with ShutdownFunc {
16 | var _context:TopologyContext = _
17 | var _collector:SpoutOutputCollector = _
18 | var _conf: JMap[_, _] = _
19 |
20 | // A constructor for the common case when you just want to output to the default stream
21 | def this(outputFields: List[String], distributed: Boolean = false) = this(collection.Map("default" -> outputFields), distributed)
22 |
23 | def open(conf: JMap[_, _], context: TopologyContext, collector: SpoutOutputCollector) = {
24 | _context = context
25 | _collector = collector
26 | _conf = conf
27 | _setup()
28 | }
29 |
30 | override def close() = {
31 | _cleanup()
32 | }
33 |
34 | // nextTuple needs to be defined by each spout inheriting from here
35 | //def nextTuple() {}
36 |
37 | def declareOutputFields(declarer: OutputFieldsDeclarer) =
38 | streamToFields foreach { case(stream, fields) =>
39 | declarer.declareStream(stream, new Fields(fields:_*))
40 | }
41 |
42 | // DSL for emit and emitDirect.
43 | // [toStream()] emit (val1, val2, ..)
44 | // [using][msgId ] [toStream ] emit (val1, val2, ...)
45 | // [using][msgId ] [toStream ] emitDirect (taskId, val1, val2, ...)
46 | def using = this
47 |
48 | def msgId(messageId: Any) = new MessageIdEmitter(_collector, messageId.asInstanceOf[AnyRef])
49 |
50 | def toStream(streamId: String) = new StreamEmitter(_collector, streamId)
51 |
52 | // Autoboxing is done for both emit and emitDirect
53 | def emit(values: Any*) = _collector.emit(values.toList.map { _.asInstanceOf[AnyRef] })
54 |
55 | def emitDirect(taskId: Int, values: Any*) = _collector.emitDirect(taskId,
56 | values.toList.map { _.asInstanceOf[AnyRef] })
57 | }
58 |
59 |
60 | class StreamEmitter(collector: SpoutOutputCollector, streamId: String) {
61 | def emit(values: AnyRef*) = collector.emit(streamId, values.toList)
62 |
63 | def emitDirect(taskId: Int, values: AnyRef*) = collector.emitDirect(taskId, streamId, values.toList)
64 | }
65 |
66 |
67 | class MessageIdEmitter(collector: SpoutOutputCollector, msgId: AnyRef) {
68 | var emitFunc: List[AnyRef] => Seq[java.lang.Integer] = collector.emit(_, msgId).asScala
69 | var emitDirectFunc: (Int, List[AnyRef]) => Unit = collector.emitDirect(_, _, msgId)
70 |
71 | def toStream(streamId: String) = {
72 | emitFunc = collector.emit(streamId, _, msgId)
73 | emitDirectFunc = collector.emitDirect(_, streamId, _, msgId)
74 | this
75 | }
76 |
77 | def emit(values: AnyRef*) = emitFunc(values.toList)
78 |
79 | def emitDirect(taskId: Int, values: AnyRef*) = emitDirectFunc(taskId, values.toList)
80 | }
81 |
--------------------------------------------------------------------------------
/src/main/scala/storm/scala/dsl/StormTuple.scala:
--------------------------------------------------------------------------------
1 | // Copyright (c) 2011 Evan Chan
2 |
3 | package storm.scala.dsl
4 |
5 | import org.apache.storm.tuple.{Fields, Tuple, Values}
6 | import org.apache.storm.task.TopologyContext
7 | import org.apache.storm.task.OutputCollector
8 | import org.apache.storm.tuple.MessageId
9 | import collection.JavaConversions._
10 | import collection.JavaConverters._
11 |
12 |
13 | // A base class for the other DSL classes
14 | abstract class BaseEmitDsl(val collector: OutputCollector) {
15 | var emitFunc: List[AnyRef] => Seq[java.lang.Integer] = collector.emit(_).asScala
16 | var emitDirectFunc: (Int, List[AnyRef]) => Unit = collector.emitDirect(_, _)
17 |
18 | // The emit function takes in a variable list of (arg1, arg2, ...) which looks
19 | // like a tuple! Autoboxing is done.
20 | // It returns a Seq of java.lang.Integers.
21 | def emit(values: Any*) = emitFunc(values.toList.map { _.asInstanceOf[AnyRef] })
22 |
23 | // emitDirect is for emitting directly to a specific taskId.
24 | def emitDirect(taskId: Int, values: Any*) = emitDirectFunc(taskId,
25 | values.toList.map { _.asInstanceOf[AnyRef] })
26 | }
27 |
28 |
29 | // unanchored emit:
30 | // new UnanchoredEmit(collector) emit (val1, val2, ...)
31 | // unanchored emit to a specific stream:
32 | // new UnanchoredEmit(collector) toStream emit (va1, val2, ..)
33 | class UnanchoredEmit(collector: OutputCollector) extends BaseEmitDsl(collector) {
34 | def toStream(streamId: String) = {
35 | emitFunc = collector.emit(streamId, _).asScala
36 | emitDirectFunc = collector.emitDirect(_, streamId, _)
37 | this
38 | }
39 | }
40 |
41 |
42 | // A class/DSL for emitting anchored on a single storm tuple, and acking a tuple.
43 | // emit anchored on one StormTuple:
44 | // stormTuple emit (val1, val2, .. )
45 | // emit anchored on one StormTuple for a stream:
46 | // stormTuple toStream emit (val1, val2, ...)
47 | class StormTuple(collector: OutputCollector, val tuple:Tuple)
48 | extends BaseEmitDsl(collector) {
49 | // Default emit function to one that takes in the tuple as the anchor
50 | emitFunc = collector.emit(tuple, _).asScala
51 | emitDirectFunc = collector.emitDirect(_, tuple, _)
52 |
53 | // stream method causes the emit to emit to a specific stream
54 | def toStream(streamId: String) = {
55 | emitFunc = collector.emit(streamId, tuple, _).asScala
56 | emitDirectFunc = collector.emitDirect(_, streamId, tuple, _)
57 | this
58 | }
59 |
60 | // Ack this tuple
61 | def ack = collector.ack(tuple)
62 |
63 | // Fail this tuple
64 | def fail = collector.fail(tuple)
65 |
66 | val lastResort: PartialFunction[Seq[Any], Unit] = {
67 | case _ => throw new RuntimeException("Unhandled tuple " + tuple)
68 | }
69 |
70 | // Use Scala pattern matching on Storm tuples!
71 | // Pass a partial function to this method with case Seq(..)
72 | // statements. Scala will match up any primitives correctly
73 | // with their boxed java.lang.Object types in the tuple.
74 | // Anything not matched by the partial function will result
75 | // in an exception.
76 | def matchSeq(f: PartialFunction[Seq[Any], Unit]) = {
77 | val matchFunc = f orElse lastResort
78 | matchFunc(tuple.getValues.asScala: Seq[Any])
79 | }
80 | }
81 |
82 |
83 | // A class/DSL for emitting anchored on multiple tuples
84 | //
85 | // multi-anchored emit:
86 | // List(tuple1,tuple2) emit (val1, val2, ...)
87 | class StormTupleList(collector: OutputCollector, val tuples: Seq[Tuple])
88 | extends BaseEmitDsl(collector) {
89 |
90 | emitFunc = collector.emit(tuples, _).asScala
91 | emitDirectFunc = collector.emitDirect(_, tuples, _)
92 |
93 | // There is no interface for emitting to a specific stream anchored on multiple tuples.
94 |
95 | // convenience func for acking a list of tuples
96 | def ack = tuples foreach { collector.ack }
97 |
98 | // convenience func for failing a list of tuples
99 | def fail = tuples foreach { collector.fail }
100 | }
101 |
--------------------------------------------------------------------------------
/src/test/scala/storm/scala/examples/AggregationTopology.scala:
--------------------------------------------------------------------------------
1 | package storm.scala.examples
2 |
3 | import storm.scala.dsl._
4 | import org.apache.storm.Config
5 | import org.apache.storm.LocalCluster
6 | import org.apache.storm.topology.TopologyBuilder
7 | import collection.mutable.{ListBuffer, HashMap}
8 | import util.Random
9 | import org.apache.storm.tuple.{Fields, Tuple}
10 | import scala.language.postfixOps
11 |
12 | /*
13 | * This is an example of streaming aggregation for different web metrics.
14 | * URLs are coming in with different geo information and devicetypes.
15 | *
16 | * It shows off the use of different streams, autoboxing and unboxing
17 | * in both spouts and bolts, and the use of a clock spout for controlling
18 | * aggregation. Also, it demonstrates how to do acking of multiple tuples,
19 | * and emitting with a message ID to get failure detection.
20 | */
21 |
22 | class WebRequestSpout extends StormSpout(outputFields = List("url", "city", "browser")) {
23 | val Urls = List("http://www.apple.com/specials", "http://itunes.apple.com/Xk#$kX",
24 | "http://hockey.espn.com", "http://oaklandathletics.com", "http://www.espn3.com/today/",
25 | "http://www.homedepot.com/Kitchen/2012/02/11")
26 | val Cities = List("San Jose", "San Francisco", "New York", "Dallas", "Chicago", "Chattanooga")
27 | val Browsers = List("IE7", "IE8", "IE9", "Firefox4", "Chrome", "Safari", "Mobile Safari", "Android Browser")
28 |
29 | def nextTuple {
30 | Thread sleep 100
31 | // Emitting with a unique message ID allows Storm to detect failure and resend tuples
32 | using msgId(Random.nextInt) emit (Urls(Random.nextInt(Urls.length)),
33 | Cities(Random.nextInt(Cities.length)),
34 | Browsers(Random.nextInt(Browsers.length)) )
35 | }
36 | }
37 |
38 | // This controls the interval at which aggregation buckets roll over
39 | class ClockSpout extends StormSpout(outputFields = List("timestamp")) {
40 | def nextTuple {
41 | Thread sleep 5000
42 | emit (System.currentTimeMillis / 1000)
43 | }
44 | }
45 |
46 |
47 | // Split original stream into three
48 | class Splitter extends StormBolt(Map("city" -> List("city"), "browser" -> List("browser"))) {
49 | def execute(t: Tuple) {
50 | t matchSeq {
51 | case Seq(url: String, city: String, browser: String) =>
52 | using anchor t toStream "city" emit (city)
53 | using anchor t toStream "browser" emit (browser)
54 | t ack
55 | }
56 | }
57 | }
58 |
59 |
60 | class FieldAggregator(val fieldName: String) extends StormBolt(outputFields = List(fieldName, "count")) {
61 | var tuples: ListBuffer[Tuple] = _
62 | var fieldCounts: collection.mutable.Map[String, Int] = _
63 | setup {
64 | tuples = new ListBuffer[Tuple]
65 | fieldCounts = new HashMap[String, Int].withDefaultValue(0)
66 | }
67 |
68 | // Only ack when the interval is complete, and ack/anchor all tuples
69 | def execute(t: Tuple) {
70 | t matchSeq {
71 | case Seq(field: String) =>
72 | fieldCounts(field) += 1
73 | tuples += t
74 | case Seq(clockTime: Long) =>
75 | fieldCounts foreach { case(field, count) =>
76 | tuples emit (field, count)
77 | }
78 | tuples.ack
79 | fieldCounts.clear
80 | tuples.clear
81 | }
82 | }
83 | }
84 |
85 | class CityAggregator extends FieldAggregator("city")
86 | class BrowserAggregator extends FieldAggregator("browser")
87 |
88 |
89 | object AggregationTopology {
90 | def main(args: Array[String]) = {
91 | val builder = new TopologyBuilder
92 |
93 | builder.setSpout("webrequest", new WebRequestSpout, 3)
94 | builder.setSpout("clock", new ClockSpout)
95 |
96 | builder.setBolt("splitter", new Splitter, 3)
97 | .shuffleGrouping("webrequest")
98 | builder.setBolt("city_aggregator", new CityAggregator, 5)
99 | .fieldsGrouping("splitter", "city", new Fields("city"))
100 | .allGrouping("clock")
101 | builder.setBolt("browser_aggregator", new BrowserAggregator, 5)
102 | .fieldsGrouping("splitter", "browser", new Fields("browser"))
103 | .allGrouping("clock")
104 |
105 | val conf = new Config
106 | conf.setDebug(true)
107 | conf.setMaxTaskParallelism(3)
108 |
109 | val cluster = new LocalCluster
110 | cluster.submitTopology("webaggregation", conf, builder.createTopology)
111 | Thread sleep 10000
112 | cluster.shutdown
113 | }
114 | }
115 |
--------------------------------------------------------------------------------
/src/test/scala/storm/scala/test/MetricsTopologyTest.scala:
--------------------------------------------------------------------------------
1 | package storm.scala.test
2 |
3 | import java.util.{Map => JMap}
4 |
5 | import org.apache.storm.metric.api.IMetricsConsumer.{DataPoint, TaskInfo}
6 | import org.apache.storm.{LocalCluster, Config}
7 | import org.apache.storm.metric.api.CountMetric
8 | import org.apache.storm.topology.TopologyBuilder
9 | import org.apache.storm.tuple.Tuple
10 | import org.scalatest._
11 | import storm.scala.dsl.{StormMetricsConsumer, StormBolt}
12 | import collection.JavaConverters._
13 |
14 | /**
15 | * registers the metric processor to only handle the 'metric:1' metric
16 | */
17 | class TestMetricsConsumer extends StormMetricsConsumer(allowedMetrics = Set("metric:1")) with ShouldMatchers {
18 | var count = 0
19 |
20 | setup {
21 | _argConfig should be (None) // we did not provide any arguments for this MetricsConsumer
22 | _conf should not be null
23 | _context should not be null
24 | _errorReporter should not be null
25 | }
26 |
27 | /** process the metric */
28 | override def processDataPoints(taskInfo: TaskInfo, dataPoints: Set[DataPoint]): Unit = {
29 | dataPoints should not be 'empty
30 |
31 | // 'metric:2' should not be received here as we did not put it in our allowedMetrics
32 | dataPoints.foreach (_.name should be ("metric:1"))
33 | count += 1
34 | }
35 |
36 | shutdown {
37 | // we should have received some metrics
38 | count should be > 0
39 | }
40 | }
41 |
42 | /**
43 | * registers the metric processor to handle all metrics
44 | */
45 | class TestMetricsConsumerWithConfig extends StormMetricsConsumer() with ShouldMatchers {
46 | var count = 0
47 |
48 | setup {
49 | _argConfig should be ('defined)
50 | _argConfig.get should (contain key "key" and contain value "value")
51 |
52 | _conf should not be null
53 | _context should not be null
54 | _errorReporter should not be null
55 | }
56 |
57 | /** process the metric */
58 | override def processDataPoints(taskInfo: TaskInfo, dataPoints: Set[DataPoint]): Unit = {
59 | dataPoints should not be 'empty
60 |
61 | // all metrics should be received here as we put nothing in our allowedMetrics list
62 | dataPoints.foreach (dp => Set("metric:1", "metric:2") should contain (dp.name))
63 | count += 1
64 | }
65 |
66 | shutdown {
67 | // we should have received some metrics
68 | count should be > 0
69 | }
70 | }
71 |
72 | /**
73 | * a test bolt that just increments the 'metric:1' and 'metric:2' metric on each tick tuple received
74 | */
75 | class TestBolt extends StormBolt(outputFields = List()) {
76 | var metric1 : CountMetric = _
77 | var metric2 : CountMetric = _
78 |
79 | setup {
80 | metric1 = new CountMetric()
81 | metric2 = new CountMetric()
82 | _context.registerMetric("metric:1", metric1, 1)
83 | _context.registerMetric("metric:2", metric2, 1)
84 | }
85 |
86 | override def execute(t: Tuple): Unit = {
87 | metric1.incr()
88 | metric2.incr()
89 | t.ack
90 | }
91 |
92 | override def getComponentConfiguration : JMap[String, Object] = {
93 | val interval : Int = 1
94 | Map(Config.TOPOLOGY_TICK_TUPLE_FREQ_SECS -> interval.asInstanceOf[Object]).asJava
95 | }
96 | }
97 |
98 | /**
99 | *
100 | */
101 | class MetricsTopologyTest extends FeatureSpec with GivenWhenThen with ShouldMatchers {
102 | feature("Metrics assigned within a topology should be processed by the wrapped StormMetricsConsumer DSL") {
103 | scenario("Start topology with registered metric and check if it gets offered to the metric implementation") {
104 | Given("A test topology with metric 'metric:1' and 'metric:2' registered, and running on ticks")
105 | val builder = new TopologyBuilder()
106 | builder.setBolt("tick", new TestBolt, 1)
107 |
108 | val conf = new Config()
109 | conf setDebug true
110 | conf registerMetricsConsumer(classOf[TestMetricsConsumer], 1)
111 | conf registerMetricsConsumer(classOf[TestMetricsConsumerWithConfig], Map("key" -> "value").asJava, 1)
112 |
113 | val cluster = new LocalCluster()
114 | cluster.submitTopology("MetricsTopologyTest", conf, builder.createTopology())
115 |
116 | When("a tick is seen by the bolt")
117 | Then("only 'metric:1' is offered to the registered TestMetricsConsumer")
118 | And("all metrics are offered to the registered TestMetricsConsumerWithConfig")
119 |
120 | Thread sleep 5000
121 | cluster.killTopology("MetricsTopologyTest")
122 | Thread sleep 1000
123 | cluster.shutdown()
124 | }
125 | }
126 | }
127 |
--------------------------------------------------------------------------------
/src/main/scala/storm/scala/dsl/StormMetricsConsumer.scala:
--------------------------------------------------------------------------------
1 | package storm.scala.dsl
2 |
3 | import java.util.{Map => JMap, Collection => JCollection}
4 |
5 | import org.apache.storm.metric.api.IMetricsConsumer
6 | import org.apache.storm.metric.api.IMetricsConsumer.TaskInfo
7 | import org.apache.storm.metric.api.IMetricsConsumer.DataPoint
8 | import org.apache.storm.task.{IErrorReporter, TopologyContext}
9 | import collection.JavaConverters._
10 |
11 | /**
12 | * Wrapper around the IMetricsConsumer of Storm
13 | * Classes implementing this base class need to implement the processDataPoints method
14 | * i.s.o. the default IMetricsConsumer::handleDataPoints method in order to process metric tuples.
15 | * This is done because a list of interested metric keys can be given at construction time so that only
16 | * these metrics will be provided in the overridden processDataPoints method.
17 | *
18 | * USAGE :
19 | * {{{
20 | * class MyMetricConsumer extends StormMetricsConsumer {
21 | * setup {
22 | * // all initialization should be done here
23 | * // available base properties :
24 | * // _conf : full storm configuration
25 | * // _argConfig : metric consumer specific config (as specified in the topology.metrics.consumer.register configuration property)
26 | * // _context : TopologyContext
27 | * // _errorReporter : IErrorReporter
28 | * }
29 | *
30 | * shutdown {
31 | * // all resource cleanup should be here
32 | * // available base properties :
33 | * // _conf : full storm configuration
34 | * // _argConfig : metric consumer specific config (as specified in the topology.metrics.consumer.register configuration property)
35 | * // _context : TopologyContext
36 | * // _errorReporter : IErrorReporter
37 | * }
38 | *
39 | * override protected def processDataPoints(taskInfo: TaskInfo, dataPoints: List[DataPoint]): Unit = {
40 | * //do something with the delivered data points
41 | * //e.g. forward them to your metric system (like ganglia)
42 | * }
43 | * }
44 | * }}}
45 | */
46 | abstract class StormMetricsConsumer(val allowedMetrics : Set[String]) extends IMetricsConsumer with SetupFunc with ShutdownFunc {
47 |
48 | protected var _conf: JMap[_, _] = _
49 | protected var _argConfig: Option[Map[String, _]] = _
50 | protected var _context: TopologyContext = _
51 | protected var _errorReporter: IErrorReporter = _
52 |
53 | /** default constructor
54 | * empty metric list, thus meaning that you're interested in all delivered metrices */
55 | def this() = { this(Set()) }
56 |
57 | /**
58 | * Initialize the metric consumer and call its registered setup functions
59 | *
60 | * @param conf the storm configuration
61 | * @param registrationArgument metric plugin specific config as defined in the config file (should be a key/value map)
62 | * @param context the topology context
63 | * @param errorReporter to report back possible exceptions
64 | */
65 | final override def prepare(conf: JMap[_, _], registrationArgument: AnyRef, context: TopologyContext, errorReporter: IErrorReporter) : Unit = {
66 | _context = context
67 | _errorReporter = errorReporter
68 | _conf = conf
69 | _argConfig = Option(registrationArgument.asInstanceOf[JMap[String, _]]).map(_.asScala.toMap)
70 | _setup()
71 | }
72 |
73 | /**
74 | * Derived classes should use the processDataPoints method
75 | *
76 | * @param taskInfo taskInfo of the task that originated the metric tuples
77 | * @param dataPoints the list of received data points
78 | */
79 | final override def handleDataPoints(taskInfo: TaskInfo, dataPoints: JCollection[DataPoint]) : Unit = {
80 | val dps = if (allowedMetrics.isEmpty) dataPoints.asScala.toSet
81 | else dataPoints.asScala.toSet.filter(allowedMetrics contains _.name)
82 |
83 | if (dps.nonEmpty) processDataPoints(taskInfo, dps)
84 | }
85 |
86 | /**
87 | * execute the registered cleanup functions
88 | */
89 | final override def cleanup() = _cleanup()
90 |
91 | /**
92 | * Needs to be implemented in order to be able to process the data points.
93 | * The passed list of data points is filtered when a list of interested data points
94 | * was given at construction time.
95 | *
96 | * @param taskInfo taskInfo of the task that originated the data points
97 | * @param dataPoints the list of received data points
98 | */
99 | protected def processDataPoints(taskInfo: TaskInfo, dataPoints: Set[DataPoint]) : Unit
100 | }
101 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | ScalaStorm provides a Scala DSL for Nathan Marz's [Storm](https://github.com/nathanmarz/storm) real-time computation system. It also provides a framework for Scala and SBT development of Storm topologies.
2 |
3 | For example, here is the SplitSentence bolt from the word count topology:
4 |
5 | ```scala
6 | class SplitSentence extends StormBolt(outputFields = List("word")) {
7 | def execute(t: Tuple) = t matchSeq {
8 | case Seq(sentence: String) => sentence split " " foreach
9 | { word => using anchor t emit (word) }
10 | t ack
11 | }
12 | }
13 | ```
14 |
15 | A couple things to note here:
16 |
17 | * The matchSeq DSL enables Scala pattern matching on Storm tuples. Notice how it gives you a nice way to name and identify the type of each component. Now imagine the ability to match on different tuple types, like in a join, easily and elegantly!
18 | * The emit DSL reads like English and easily takes multiple args (val1, val2, ...)
19 | * Output fields are easily declared
20 | * It's easy to see exactly when the emits and ack happen
21 |
22 | Useful features for Scala developers:
23 |
24 | * Auto-boxing of Scala primitives in tuple emit and matchSeq
25 | * A BoltDsl trait for using the DSL from any thread/actor/class
26 |
27 | 0.2.4
28 | =====
29 | Added support for multiple streams in Spouts:
30 | ```scala
31 | class MultiStreamSpout extends StormSpout(Map("city" -> List("city"), "browser" -> List("browser"))) {
32 | }
33 | ```
34 | Switched to Apache Storm distribution
35 | Build system updated to sbt 0.13.5
36 | Build system supports crosscompiling for scala 2.9/2.10
37 | ShutdownFunc trait added to StormSpout
38 |
39 | Please Read For 0.2.2 / Storm 0.8.0+ Users
40 | =========================================
41 | Storm 0.8.0 emits are no longer thread safe. You may see NullPointerExceptions with DisruptorQueue in the stack trace.
42 | If you are doing emits from multiple threads or actors, you will need to synchronize your emits or have them
43 | come from a single thread. You should synchronize on the collector instance:
44 |
45 | ```scala
46 | _collector.synchronized { tuple emit (val1, val2) }
47 | ```
48 |
49 | ## Functional Trident (NEW!)
50 |
51 | There is a sample Trident topology, in src/storm/scala/examples/trident. It features an
52 | experimental new DSL for doing functional Trident topologies (see FunctionalTrident.scala). I am
53 | currently soliciting feedback for this feature, so drop me a line if you like it.
54 |
55 | Getting Started
56 | ===============
57 |
58 | The latest version of scala-storm, 0.2.2, corresponds to Storm 0.8.1 and is available from Maven Central. Add this to your build.sbt:
59 |
60 | ```scala
61 | libraryDependencies += "com.github.velvia" %% "scala-storm" % "0.2.2"
62 | ```
63 |
64 | Version 0.2.0 is available from Maven central and corresponds to Storm 0.7.1.
65 | ```scala
66 | libraryDependencies += "com.github.velvia" %% "scala-storm" % "0.2.0"
67 | ```
68 |
69 | In both cases, you will need additional repos, as maven central does not host the Storm/clojure jars:
70 | ```scala
71 | resolvers ++= Seq("clojars" at "http://clojars.org/repo/",
72 | "clojure-releases" at "http://build.clojure.org/releases")
73 | ```
74 |
75 | If you want to build from source:
76 |
77 | * Download [sbt](https://github.com/harrah/xsbt/wiki) version 0.10.1 or above
78 | * clone this project
79 | * In the root project dir, type `sbt test:run`. SBT will automatically download all dependencies, compile the code, and give you a menu of topologies to run.
80 |
81 | To help you get started, the ExclamationTopology and WordCountTopology examples from storm starter have been included.
82 |
83 | Bolt DSL
84 | ========
85 | The Scala DSL for bolts is designed to support many different bolt designs, including all 10 variants of the collector emit() and emitDirect() APIs. Getting started consists of extending the StormBolt class, passing a list of output fields, and defining the execute method:
86 |
87 | ```scala
88 | class ExclamationBolt extends StormBolt(outputFields = List("word")) {
89 | def execute(t: Tuple) = {
90 | t emit (t.getString(0) + "!!!")
91 | t ack
92 | }
93 | }
94 | ```
95 |
96 | If you need to emit to multiple output streams, that can be done by passing a Map with the key being the stream name/Id, and the value being the list of fields for each stream (See the AggregationTopology example):
97 | ```scala
98 | class Splitter extends StormBolt(Map("city" -> List("city"), "browser" -> List("browser"))) {
99 | }
100 | ```
101 |
102 | BoltDsl trait
103 | -------------
104 | If you want to use the emit DSL described below in a thread or Actor, you can use the BoltDsl trait. You just have to initialise the _collector variable.
105 |
106 | ```scala
107 | class DataWorker(val collector: OutputCollector) extends Actor with BoltDsl {
108 | _collector = collector
109 | ...
110 | def receive = {
111 | no anchor emit (someString, someInt)
112 | }
113 | }
114 | ```
115 |
116 | matchSeq
117 | --------
118 | The `matchSeq` method passes the storm tuple as a Scala Seq to the given code block with one or more case statements. The case statements need to have Seq() in order to match the tuple. If none of the cases match, then by default a handler which throws a RuntimeError will be used. It is a good idea to include your own default handler.
119 |
120 | matchSeq allows easy naming and safe typing of tuple components, and allows easy parsing of different tuple types. Suppose that a bolt takes in a data stream from one source and a clock or timing-related stream from another source. It can be handled like this:
121 |
122 | ```scala
123 | def execute(t: Tuple) = t matchSeq {
124 | case Seq(username: String, followers: List[String]) => // process data
125 | case Seq(timestamp: Integer) => // process clock event
126 | }
127 | ```
128 |
129 | Unboxing will be automatically performed. Even though everything going over the wire has to be a subset of java.lang.Object, if you match on a Scala primitive, it will automatically unbox it for you.
130 |
131 | By default, if none of the cases are matched, then ScalaStorm will throw a RuntimeException with a message "unhandled tuple". This can be useful for debugging in local mode to quickly discover matching errors. If you want to handle the unhandled case yourself, simply add `case _ => ...` as the last case.
132 |
133 | emit and emitDirect
134 | -------------------
135 | emit takes a variable number of AnyRef arguments which make up the tuple to emit. emitDirect is the same but the first argument is the Int taskId, followed by a variable number of AnyRefs.
136 |
137 | To emit a tuple anchored on one tuple, where t is of type Tuple, do one of the following:
138 |
139 | ```scala
140 | using anchor t emit (val1, val2, ...)
141 | using anchor t emitDirect (taskId, val1, val2, ...)
142 | anchor(t) emit (val1, val2, ...)
143 | t emit (val1, val2, ...)
144 | ```
145 |
146 | To emit a tuple to a particular stream:
147 |
148 | ```scala
149 | using anchor t toStream 5 emit (val1, val2, ...)
150 | using anchor t toStream 5 emitDirect (taskId, val1, val2, ...)
151 | ```
152 |
153 | To emit anchored on multiple tuples (can be any Seq, not just a List):
154 |
155 | ```scala
156 | using anchor List(t1, t2) emit (val1, val2, ...)
157 | using anchor List(t1, t2) emitDirect (taskId, val1, val2, ...)
158 | ```
159 |
160 | To emit unanchored:
161 |
162 | ```scala
163 | using no anchor emit (val1, val2, ...)
164 | using no anchor emitDirect (taskId, val1, val2, ...)
165 | using no anchor toStream 5 emit (val1, val2, ...)
166 | using no anchor toStream 5 emitDirect (taskId, val1, val2, ...)
167 | ```
168 |
169 | ack
170 | ---
171 | ```scala
172 | t ack // Ack one tuple
173 | List(t1, t2) ack // Ack multiple tuples, in order of list
174 | ```
175 |
176 | A note on types supported by emit (...)
177 | ---------------------------------------
178 | Any scala type may be passed to emit() so long as it can be autoboxed into an AnyRef (java.lang.Object). This includes Scala Ints, Longs, and other basic types.
179 |
180 | Spout DSL
181 | =========
182 | The Scala Spout DSL is very similar to the Bolt DSL. One extends the StormSpout class, declaring the output fields, and defines the nextTuple method:
183 |
184 | ```scala
185 | class MySpout extends StormSpout(outputFields = List("word", "author")) {
186 | def nextTuple = {}
187 | }
188 | ```
189 |
190 | Spout emit DSL
191 | --------------
192 | The spout emit DSL is very similar to the bolt emit DSL. Again, all variants of the SpoutOutputCollector emit and emitDirect APIs are supported. The basic forms for emitting tuples are as follows:
193 |
194 | ```scala
195 | emit (val1, val2, ...)
196 | emitDirect (taskId, val1, val2, ...)
197 | ```
198 |
199 | To emit a tuple with a specific message ID:
200 |
201 | ```scala
202 | using msgId 9876 emit (val1, val2, ...)
203 | using msgId 9876 emitDirect (taskId, val1, val2, ...)
204 | ```
205 |
206 | To emit a tuple to a specific stream:
207 |
208 | ```scala
209 | toStream 6 emit (val1, val2, ...)
210 | toStream 6 emitDirect (taskId, val1, val2, ...)
211 | using msgId 9876 toStream 6 emit (val1, val2, ...)
212 | using msgId 9876 toStream 6 emitDirect (taskId, val1, val2, ...)
213 | ```
214 |
215 | Bolt and Spout Setup
216 | ====================
217 | You will probably need to initialize per-instance variables at each bolt and spout for all but the simplest of designs. You should not do this in the Bolt or Spout constructor, as the constructor is only called before submitting the Topology. What you instead need to do is to override the prepare() and open() methods, and do your setup in there, but there is a convenient `setup` DSL that lets you perform whatever per-instance initialization is needed, in a concise and consistent manner. To use it:
218 |
219 | ```scala
220 | class MyBolt extends StormBolt(List("word")) {
221 | var myIterator: Iterator[Int] = _
222 | setup { myIterator = ... }
223 | }
224 | ```
225 |
226 | License
227 | =======
228 | Apache 2.0. Please see LICENSE.md.
229 | All contents copyright (c) 2012, Evan Chan.
230 |
--------------------------------------------------------------------------------