forked from twitter/summingbird
-
Notifications
You must be signed in to change notification settings - Fork 0
/
FlatMapBoltProvider.scala
172 lines (146 loc) · 6.68 KB
/
FlatMapBoltProvider.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
/*
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 Constants._
import backtype.storm.topology.TopologyBuilder
import backtype.storm.tuple.Fields
import backtype.storm.tuple.Tuple
import com.twitter.algebird.{ Semigroup, Monoid }
import com.twitter.storehaus.ReadableStore
import com.twitter.summingbird._
import com.twitter.summingbird.chill._
import com.twitter.summingbird.batch.{ BatchID, Batcher, Timestamp }
import com.twitter.summingbird.storm.option.{ AckOnEntry, AnchorTuples }
import com.twitter.summingbird.online.executor.InputState
import com.twitter.summingbird.online.option.{ IncludeSuccessHandler, MaxWaitingFutures, MaxFutureWaitTime, SummerBuilder }
import com.twitter.summingbird.option.{ CacheSize, JobId }
import com.twitter.summingbird.planner._
import com.twitter.summingbird.online.executor
import com.twitter.summingbird.online.FlatMapOperation
import com.twitter.summingbird.storm.planner._
import org.slf4j.LoggerFactory
import scala.collection.{ Map => CMap }
/**
* These are helper functions for building a bolt from a Node[Storm] element.
* There are two main codepaths here, for intermediate flat maps and final flat maps.
* The primary difference between those two being the the presents of map side aggreagtion in a final flatmap.
*/
object FlatMapBoltProvider {
@transient private val logger = LoggerFactory.getLogger(FlatMapBoltProvider.getClass)
private def wrapTimeBatchIDKV[T, K, V](existingOp: FlatMapOperation[T, (K, V)])(batcher: Batcher): FlatMapOperation[(Timestamp, T), ((K, BatchID), (Timestamp, V))] = {
FlatMapOperation.generic({
case (ts: Timestamp, data: T) =>
existingOp.apply(data).map { vals =>
vals.map { tup =>
((tup._1, batcher.batchOf(ts)), (ts, tup._2))
}
}
})
}
def wrapTime[T, U](existingOp: FlatMapOperation[T, U]): FlatMapOperation[(Timestamp, T), (Timestamp, U)] = {
FlatMapOperation.generic({ x: (Timestamp, T) =>
existingOp.apply(x._2).map { vals =>
vals.map((x._1, _))
}
})
}
}
case class FlatMapBoltProvider(storm: Storm, jobID: JobId, stormDag: Dag[Storm], node: StormNode)(implicit topologyBuilder: TopologyBuilder) {
import FlatMapBoltProvider._
import Producer2FlatMapOperation._
def getOrElse[T <: AnyRef: Manifest](default: T, queryNode: StormNode = node) = storm.getOrElse(stormDag, queryNode, default)
// Boilerplate extracting of the options from the DAG
private val nodeName = stormDag.getNodeName(node)
private val metrics = getOrElse(DEFAULT_FM_STORM_METRICS)
private val anchorTuples = getOrElse(AnchorTuples.default)
logger.info("[{}] Anchoring: {}", nodeName, anchorTuples.anchor)
private val maxWaiting = getOrElse(DEFAULT_MAX_WAITING_FUTURES)
private val maxWaitTime = getOrElse(DEFAULT_MAX_FUTURE_WAIT_TIME)
logger.info("[{}] maxWaiting: {}", nodeName, maxWaiting.get)
private val ackOnEntry = getOrElse(DEFAULT_ACK_ON_ENTRY)
logger.info("[{}] ackOnEntry : {}", nodeName, ackOnEntry.get)
val maxExecutePerSec = getOrElse(DEFAULT_MAX_EXECUTE_PER_SEC)
logger.info("[{}] maxExecutePerSec : {}", nodeName, maxExecutePerSec.toString)
private val maxEmitPerExecute = getOrElse(DEFAULT_MAX_EMIT_PER_EXECUTE)
logger.info("[{}] maxEmitPerExecute : {}", nodeName, maxEmitPerExecute.get)
private def getFFMBolt[T, K, V](summer: SummerNode[Storm]) = {
type ExecutorInput = (Timestamp, T)
type ExecutorKey = Int
type InnerValue = (Timestamp, V)
type ExecutorValue = CMap[(K, BatchID), InnerValue]
val summerProducer = summer.members.collect { case s: Summer[_, _, _] => s }.head.asInstanceOf[Summer[Storm, K, V]]
// When emitting tuples between the Final Flat Map and the summer we encode the timestamp in the value
// The monoid we use in aggregation is timestamp max.
val batcher = summerProducer.store.batcher
implicit val valueMonoid: Semigroup[V] = summerProducer.semigroup
// Query to get the summer paralellism of the summer down stream of us we are emitting to
// to ensure no edge case between what we might see for its parallelism and what it would see/pass to storm.
val summerParalellism = getOrElse(DEFAULT_SUMMER_PARALLELISM, summer)
val summerBatchMultiplier = getOrElse(DEFAULT_SUMMER_BATCH_MULTIPLIER, summer)
// This option we report its value here, but its not user settable.
val keyValueShards = executor.KeyValueShards(summerParalellism.parHint * summerBatchMultiplier.get)
logger.info("[{}] keyValueShards : {}", nodeName, keyValueShards.get)
val operation = foldOperations[T, (K, V)](node.members.reverse)
val wrappedOperation = wrapTimeBatchIDKV(operation)(batcher)
val builder = BuildSummer(storm, stormDag, node, jobID)
BaseBolt(
jobID,
metrics.metrics,
anchorTuples,
true,
new Fields(AGG_KEY, AGG_VALUE),
ackOnEntry,
maxExecutePerSec,
new executor.FinalFlatMap(
wrappedOperation,
builder,
maxWaiting,
maxWaitTime,
maxEmitPerExecute,
keyValueShards,
new SingleItemInjection[ExecutorInput],
new KeyValueInjection[ExecutorKey, ExecutorValue]
)(implicitly[Semigroup[InnerValue]])
)
}
def getIntermediateFMBolt[T, U] = {
type ExecutorInput = (Timestamp, T)
type ExecutorOutput = (Timestamp, U)
val operation = foldOperations[T, U](node.members.reverse)
val wrappedOperation = wrapTime(operation)
BaseBolt(
jobID,
metrics.metrics,
anchorTuples,
stormDag.dependantsOf(node).size > 0,
new Fields(VALUE_FIELD),
ackOnEntry,
maxExecutePerSec,
new executor.IntermediateFlatMap(
wrappedOperation,
maxWaiting,
maxWaitTime,
maxEmitPerExecute,
new SingleItemInjection[ExecutorInput],
new SingleItemInjection[ExecutorOutput]
)
)
}
def apply: BaseBolt[Any, Any] = {
val summerOpt: Option[SummerNode[Storm]] = stormDag.dependantsOf(node).collect { case s: SummerNode[Storm] => s }.headOption
summerOpt match {
case Some(s) => getFFMBolt[Any, Any, Any](s).asInstanceOf[BaseBolt[Any, Any]]
case None => getIntermediateFMBolt[Any, Any].asInstanceOf[BaseBolt[Any, Any]]
}
}
}