forked from twitter/summingbird
-
Notifications
You must be signed in to change notification settings - Fork 0
/
StormPlatform.scala
362 lines (288 loc) · 15.2 KB
/
StormPlatform.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
/*
Copyright 2013 Twitter, Inc.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT 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 com.twitter.summingbird.storm
import backtype.storm.{ Config => BacktypeStormConfig, LocalCluster, StormSubmitter }
import backtype.storm.generated.StormTopology
import backtype.storm.task.TopologyContext
import backtype.storm.topology.{ BoltDeclarer, TopologyBuilder }
import backtype.storm.tuple.Fields
import com.twitter.algebird.{ Monoid, Semigroup }
import com.twitter.bijection.{ Base64String, Injection }
import com.twitter.chill.IKryoRegistrar
import com.twitter.storehaus.algebra.{ MergeableStore, Mergeable, StoreAlgebra }
import com.twitter.storehaus.{ ReadableStore, WritableStore, Store }
import com.twitter.summingbird._
import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp }
import com.twitter.summingbird.chill.SBChillRegistrar
import com.twitter.summingbird.online._
import com.twitter.summingbird.online.option._
import com.twitter.summingbird.option.JobId
import com.twitter.summingbird.planner.{ Dag, DagOptimizer, OnlinePlan, SummerNode, FlatMapNode, SourceNode }
import com.twitter.summingbird.storm.option.{ AckOnEntry, AnchorTuples }
import com.twitter.summingbird.storm.planner.StormNode
import com.twitter.summingbird.viz.VizGraph
import com.twitter.tormenta.spout.Spout
import com.twitter.util.{ Future, Time }
import org.slf4j.LoggerFactory
import scala.collection.{ Map => CMap }
import Constants._
/*
* Batchers are used for partial aggregation. We never aggregate past two items which are not in the same batch.
* This is needed/used everywhere we partially aggregate, summer's into stores, map side partial aggregation before summers, etc..
*/
sealed trait StormSource[+T]
case class SpoutSource[+T](spout: Spout[(Timestamp, T)], parallelism: Option[SourceParallelism]) extends StormSource[T]
object Storm {
def local(options: Map[String, Options] = Map.empty): LocalStorm =
new LocalStorm(options, identity, List())
def remote(options: Map[String, Options] = Map.empty): RemoteStorm =
new RemoteStorm(options, identity, List())
/**
* Below are factory methods for the input output types:
*/
def sink[T](fn: => T => Future[Unit]): Storm#Sink[T] = new SinkFn(fn)
def sinkIntoWritable[K, V](store: => WritableStore[K, V]): Storm#Sink[(K, V)] =
new WritableStoreSink[K, V](store)
// This can be used in jobs that do not have a batch component
def onlineOnlyStore[K, V](store: => MergeableStore[K, V]): MergeableStoreFactory[(K, BatchID), V] =
MergeableStoreFactory.fromOnlineOnly(store)
def store[K, V](store: => Mergeable[(K, BatchID), V])(implicit batcher: Batcher): MergeableStoreFactory[(K, BatchID), V] =
MergeableStoreFactory.from(store)
def service[K, V](serv: => ReadableStore[K, V]): ReadableServiceFactory[K, V] = ReadableServiceFactory(() => serv)
def toStormSource[T](spout: Spout[T],
defaultSourcePar: Option[Int] = None)(implicit timeOf: TimeExtractor[T]): StormSource[T] =
SpoutSource(spout.map(t => (Timestamp(timeOf(t)), t)), defaultSourcePar.map(SourceParallelism(_)))
implicit def spoutAsStormSource[T](spout: Spout[T])(implicit timeOf: TimeExtractor[T]): StormSource[T] =
toStormSource(spout, None)(timeOf)
def source[T](spout: Spout[T],
defaultSourcePar: Option[Int] = None)(implicit timeOf: TimeExtractor[T]): Producer[Storm, T] =
Producer.source[Storm, T](toStormSource(spout, defaultSourcePar))
implicit def spoutAsSource[T](spout: Spout[T])(implicit timeOf: TimeExtractor[T]): Producer[Storm, T] =
source(spout, None)(timeOf)
}
case class PlannedTopology(config: BacktypeStormConfig, topology: StormTopology)
abstract class Storm(options: Map[String, Options], transformConfig: SummingbirdConfig => SummingbirdConfig, passedRegistrars: List[IKryoRegistrar]) extends Platform[Storm] {
@transient private val logger = LoggerFactory.getLogger(classOf[Storm])
type Source[+T] = StormSource[T]
type Store[-K, V] = MergeableStoreFactory[(K, BatchID), V]
type Sink[-T] = StormSink[T]
type Service[-K, +V] = OnlineServiceFactory[K, V]
type Plan[T] = PlannedTopology
private type Prod[T] = Producer[Storm, T]
private[storm] def get[T <: AnyRef: Manifest](dag: Dag[Storm], node: StormNode): Option[(String, T)] = {
val producer = node.members.last
val namedNodes = dag.producerToPriorityNames(producer)
(for {
id <- namedNodes :+ "DEFAULT"
stormOpts <- options.get(id)
option <- stormOpts.get[T]
} yield (id, option)).headOption
}
private[storm] def getOrElse[T <: AnyRef: Manifest](dag: Dag[Storm], node: StormNode, default: T): T = {
get[T](dag, node) match {
case None =>
logger.debug("Node ({}): Using default setting {}", dag.getNodeName(node), default)
default
case Some((namedSource, option)) =>
logger.info("Node {}: Using {} found via NamedProducer \"{}\"", Array[AnyRef](dag.getNodeName(node), option, namedSource))
option
}
}
/**
* Set storm to tick our nodes every second to clean up finished futures
*/
private def tickConfig = {
val boltConfig = new BacktypeStormConfig
boltConfig.put(BacktypeStormConfig.TOPOLOGY_TICK_TUPLE_FREQ_SECS, java.lang.Integer.valueOf(1))
boltConfig
}
private def scheduleFlatMapper(jobID: JobId, stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) = {
val nodeName = stormDag.getNodeName(node)
val usePreferLocalDependency = getOrElse(stormDag, node, DEFAULT_FM_PREFER_LOCAL_DEPENDENCY)
logger.info("[{}] usePreferLocalDependency: {}", nodeName, usePreferLocalDependency.get)
val bolt: BaseBolt[Any, Any] = FlatMapBoltProvider(this, jobID, stormDag, node).apply
val parallelism = getOrElse(stormDag, node, DEFAULT_FM_PARALLELISM).parHint
val declarer = topologyBuilder.setBolt(nodeName, bolt, parallelism).addConfigurations(tickConfig)
val dependenciesNames = stormDag.dependenciesOf(node).collect { case x: StormNode => stormDag.getNodeName(x) }
if (usePreferLocalDependency.get) {
dependenciesNames.foreach { declarer.localOrShuffleGrouping(_) }
} else {
dependenciesNames.foreach { declarer.shuffleGrouping(_) }
}
}
private def scheduleSpout[K](stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) = {
val (spout, parOpt) = node.members.collect { case Source(SpoutSource(s, parOpt)) => (s, parOpt) }.head
val nodeName = stormDag.getNodeName(node)
val tormentaSpout = node.members.reverse.foldLeft(spout.asInstanceOf[Spout[(Timestamp, Any)]]) { (spout, p) =>
p match {
case Source(_) => spout // The source is still in the members list so drop it
case OptionMappedProducer(_, op) => spout.flatMap { case (time, t) => op.apply(t).map { x => (time, x) } }
case NamedProducer(_, _) => spout
case IdentityKeyedProducer(_) => spout
case AlsoProducer(_, _) => spout
case _ => sys.error("not possible, given the above call to span.\n" + p)
}
}
val metrics = getOrElse(stormDag, node, DEFAULT_SPOUT_STORM_METRICS)
val stormSpout = tormentaSpout.registerMetrics(metrics.toSpoutMetrics).getSpout
val parallelism = getOrElse(stormDag, node, parOpt.getOrElse(DEFAULT_SOURCE_PARALLELISM)).parHint
topologyBuilder.setSpout(nodeName, stormSpout, parallelism)
}
private def scheduleSummerBolt[K, V](jobID: JobId, stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) = {
val summer: Summer[Storm, K, V] = node.members.collect { case c: Summer[Storm, K, V] => c }.head
implicit val semigroup = summer.semigroup
implicit val batcher = summer.store.batcher
val nodeName = stormDag.getNodeName(node)
type ExecutorKeyType = (K, BatchID)
type ExecutorValueType = (Timestamp, V)
type ExecutorOutputType = (Timestamp, (K, (Option[V], V)))
val supplier: MergeableStoreFactory[ExecutorKeyType, V] = summer.store match {
case m: MergeableStoreFactory[ExecutorKeyType, V] => m
case _ => sys.error("Should never be able to get here, looking for a MergeableStoreFactory from %s".format(summer.store))
}
val wrappedStore: MergeableStoreFactory[ExecutorKeyType, ExecutorValueType] =
MergeableStoreFactoryAlgebra.wrapOnlineFactory(supplier)
val anchorTuples = getOrElse(stormDag, node, AnchorTuples.default)
val metrics = getOrElse(stormDag, node, DEFAULT_SUMMER_STORM_METRICS)
val shouldEmit = stormDag.dependantsOf(node).size > 0
val builder = BuildSummer(this, stormDag, node, jobID)
val ackOnEntry = getOrElse(stormDag, node, DEFAULT_ACK_ON_ENTRY)
logger.info("[{}] ackOnEntry : {}", nodeName, ackOnEntry.get)
val maxEmitPerExecute = getOrElse(stormDag, node, DEFAULT_MAX_EMIT_PER_EXECUTE)
logger.info("[{}] maxEmitPerExecute : {}", nodeName, maxEmitPerExecute.get)
val maxExecutePerSec = getOrElse(stormDag, node, DEFAULT_MAX_EXECUTE_PER_SEC)
logger.info("[{}] maxExecutePerSec : {}", nodeName, maxExecutePerSec.toString)
val storeBaseFMOp = { op: (ExecutorKeyType, (Option[ExecutorValueType], ExecutorValueType)) =>
val ((k, batchID), (optiVWithTS, (ts, v))) = op
val optiV = optiVWithTS.map(_._2)
List((ts, (k, (optiV, v))))
}
val flatmapOp: FlatMapOperation[(ExecutorKeyType, (Option[ExecutorValueType], ExecutorValueType)), ExecutorOutputType] =
FlatMapOperation.apply(storeBaseFMOp)
val sinkBolt = BaseBolt(
jobID,
metrics.metrics,
anchorTuples,
shouldEmit,
new Fields(VALUE_FIELD),
ackOnEntry,
maxExecutePerSec,
new executor.Summer(
wrappedStore,
flatmapOp,
getOrElse(stormDag, node, DEFAULT_ONLINE_SUCCESS_HANDLER),
getOrElse(stormDag, node, DEFAULT_ONLINE_EXCEPTION_HANDLER),
builder,
getOrElse(stormDag, node, DEFAULT_MAX_WAITING_FUTURES),
getOrElse(stormDag, node, DEFAULT_MAX_FUTURE_WAIT_TIME),
maxEmitPerExecute,
getOrElse(stormDag, node, IncludeSuccessHandler.default),
new KeyValueInjection[Int, CMap[ExecutorKeyType, ExecutorValueType]],
new SingleItemInjection[ExecutorOutputType])
)
val parallelism = getOrElse(stormDag, node, DEFAULT_SUMMER_PARALLELISM).parHint
val declarer =
topologyBuilder.setBolt(
nodeName,
sinkBolt,
parallelism
).addConfigurations(tickConfig)
val dependenciesNames = stormDag.dependenciesOf(node).collect { case x: StormNode => stormDag.getNodeName(x) }
dependenciesNames.foreach { parentName =>
declarer.fieldsGrouping(parentName, new Fields(AGG_KEY))
}
}
private def dumpOptions: String = {
options.map {
case (k, opts) =>
"%s -> [%s]".format(k, opts.opts.values.mkString(", "))
}.mkString("\n || ")
}
/**
* The following operations are public.
*/
/**
* Base storm config instances used by the Storm platform.
*/
def genConfig(dag: Dag[Storm]) = {
val config = new BacktypeStormConfig
config.setFallBackOnJavaSerialization(false)
config.setKryoFactory(classOf[com.twitter.chill.storm.BlizzardKryoFactory])
config.setMaxSpoutPending(1000)
config.setNumAckers(12)
config.setNumWorkers(12)
val initialStormConfig = StormConfig(config)
val stormConfig = SBChillRegistrar(initialStormConfig, passedRegistrars)
logger.debug("Serialization config changes:")
logger.debug("Removes: {}", stormConfig.removes)
logger.debug("Updates: {}", stormConfig.updates)
val inj = Injection.connect[String, Array[Byte], Base64String]
logger.debug("Adding serialized copy of graphs")
val withViz = stormConfig.put("summingbird.base64_graph.producer", inj.apply(VizGraph(dag.originalTail)).str)
.put("summingbird.base64_graph.planned", inj.apply(VizGraph(dag)).str)
val withOptions = withViz.put("summingbird.options", dumpOptions)
val transformedConfig = transformConfig(withOptions)
logger.debug("Config diff to be applied:")
logger.debug("Removes: {}", transformedConfig.removes)
logger.debug("Updates: {}", transformedConfig.updates)
transformedConfig.removes.foreach(config.remove(_))
transformedConfig.updates.foreach(kv => config.put(kv._1, kv._2))
config
}
def withRegistrars(registrars: List[IKryoRegistrar]): Storm
def withConfigUpdater(fn: SummingbirdConfig => SummingbirdConfig): Storm
def plan[T](tail: TailProducer[Storm, T]): PlannedTopology = {
/*
* TODO: storm does not yet know about ValueFlatMapped, so remove it before
* planning
*/
val dagOptimizer = new DagOptimizer[Storm] {}
val stormTail = dagOptimizer.optimize(tail, dagOptimizer.ValueFlatMapToFlatMap)
val stormDag = OnlinePlan(stormTail.asInstanceOf[TailProducer[Storm, T]])
implicit val topologyBuilder = new TopologyBuilder
implicit val config = genConfig(stormDag)
val jobID = JobId(config.get("storm.job.uniqueId").asInstanceOf[String])
stormDag.nodes.foreach { node =>
node match {
case _: SummerNode[_] => scheduleSummerBolt(jobID, stormDag, node)
case _: FlatMapNode[_] => scheduleFlatMapper(jobID, stormDag, node)
case _: SourceNode[_] => scheduleSpout(stormDag, node)
}
}
PlannedTopology(config, topologyBuilder.createTopology)
}
def run(tail: TailProducer[Storm, _], jobName: String): Unit = run(plan(tail), jobName)
def run(plannedTopology: PlannedTopology, jobName: String): Unit
}
class RemoteStorm(options: Map[String, Options], transformConfig: SummingbirdConfig => SummingbirdConfig, passedRegistrars: List[IKryoRegistrar]) extends Storm(options, transformConfig, passedRegistrars) {
override def withConfigUpdater(fn: SummingbirdConfig => SummingbirdConfig) =
new RemoteStorm(options, transformConfig.andThen(fn), passedRegistrars)
override def run(plannedTopology: PlannedTopology, jobName: String): Unit = {
val topologyName = "summingbird_" + jobName
StormSubmitter.submitTopology(topologyName, plannedTopology.config, plannedTopology.topology)
}
override def withRegistrars(registrars: List[IKryoRegistrar]) =
new RemoteStorm(options, transformConfig, passedRegistrars ++ registrars)
}
class LocalStorm(options: Map[String, Options], transformConfig: SummingbirdConfig => SummingbirdConfig, passedRegistrars: List[IKryoRegistrar])
extends Storm(options, transformConfig, passedRegistrars) {
lazy val localCluster = new LocalCluster
override def withConfigUpdater(fn: SummingbirdConfig => SummingbirdConfig) =
new LocalStorm(options, transformConfig.andThen(fn), passedRegistrars)
override def run(plannedTopology: PlannedTopology, jobName: String): Unit = {
val topologyName = "summingbird_" + jobName
localCluster.submitTopology(topologyName, plannedTopology.config, plannedTopology.topology)
}
override def withRegistrars(registrars: List[IKryoRegistrar]) =
new LocalStorm(options, transformConfig, passedRegistrars ++ registrars)
}