forked from nathanmarz/storm
/
testing.clj
471 lines (417 loc) · 17.9 KB
/
testing.clj
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
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
(ns backtype.storm.testing
(:require [backtype.storm.daemon
[nimbus :as nimbus]
[supervisor :as supervisor]
[common :as common]
[worker :as worker]
[task :as task]])
(:require [backtype.storm [process-simulator :as psim]])
(:import [org.apache.commons.io FileUtils])
(:import [java.io File])
(:import [java.util.concurrent.atomic AtomicInteger])
(:import [java.util.concurrent ConcurrentHashMap])
(:import [backtype.storm.utils Time Utils RegisteredGlobalState])
(:import [backtype.storm.tuple Fields Tuple])
(:import [backtype.storm.task TopologyContext])
(:import [backtype.storm.generated GlobalStreamId Bolt])
(:import [backtype.storm.testing FeederSpout FixedTupleSpout FixedTuple TupleCaptureBolt
SpoutTracker BoltTracker])
(:require [backtype.storm [zookeeper :as zk]])
(:require [backtype.storm.messaging.loader :as msg-loader])
(:require [backtype.storm.daemon.acker :as acker])
(:use [clojure.contrib.def :only [defnk]])
(:use [clojure.contrib.seq :only [find-first]])
(:use [backtype.storm cluster util thrift config log]))
(defn feeder-spout [fields]
(FeederSpout. (Fields. fields)))
(defn local-temp-path []
(str (System/getProperty "java.io.tmpdir") "/" (uuid)))
(defn delete-all [paths]
(dorun
(for [t paths]
(if (.exists (File. t))
(FileUtils/forceDelete (File. t))
))))
(defmacro with-local-tmp [[& tmp-syms] & body]
(let [tmp-paths (mapcat (fn [t] [t `(local-temp-path)]) tmp-syms)]
`(let [~@tmp-paths]
(try
~@body
(finally
(delete-all ~(vec tmp-syms)))
))
))
(defn start-simulating-time! []
(Time/startSimulating))
(defn stop-simulating-time! []
(Time/stopSimulating))
(defmacro with-simulated-time [& body]
`(do
(start-simulating-time!)
(let [ret# (do ~@body)]
(stop-simulating-time!)
ret#
)))
(defn advance-time-ms! [ms]
(Time/advanceTime ms))
(defn advance-time-secs! [secs]
(advance-time-ms! (* (long secs) 1000)))
(defnk add-supervisor [cluster-map :ports 2 :conf {} :id nil]
(let [tmp-dir (local-temp-path)
port-ids (if (sequential? ports) ports (doall (repeatedly ports (:port-counter cluster-map))))
supervisor-conf (merge (:daemon-conf cluster-map)
conf
{STORM-LOCAL-DIR tmp-dir
SUPERVISOR-SLOTS-PORTS port-ids
})
id-fn (if id (fn [] id) supervisor/generate-supervisor-id)
daemon (with-var-roots [supervisor/generate-supervisor-id id-fn] (supervisor/mk-supervisor supervisor-conf (:shared-context cluster-map)))]
(swap! (:supervisors cluster-map) conj daemon)
(swap! (:tmp-dirs cluster-map) conj tmp-dir)
daemon
))
(defn mk-shared-context [conf]
(if (and (= (conf STORM-CLUSTER-MODE) "local")
(not (conf STORM-LOCAL-MODE-ZMQ)))
(msg-loader/mk-local-context)
))
;; returns map containing cluster info
;; local dir is always overridden in maps
;; can customize the supervisors (except for ports) by passing in map for :supervisors parameter
;; if need to customize amt of ports more, can use add-supervisor calls afterwards
(defnk mk-local-storm-cluster [:supervisors 2 :ports-per-supervisor 3 :daemon-conf {}]
(let [zk-port (available-port 2181)
daemon-conf (merge (read-storm-config)
{TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS true
ZMQ-LINGER-MILLIS 0
}
daemon-conf
{STORM-CLUSTER-MODE "local"
STORM-ZOOKEEPER-PORT zk-port})
nimbus-tmp (local-temp-path)
zk-tmp (local-temp-path)
zk-handle (zk/mk-inprocess-zookeeper zk-tmp zk-port)
port-counter (mk-counter)
nimbus (nimbus/service-handler
(assoc daemon-conf STORM-LOCAL-DIR nimbus-tmp))
context (mk-shared-context daemon-conf)
cluster-map {:nimbus nimbus
:port-counter port-counter
:daemon-conf daemon-conf
:supervisors (atom [])
:state (mk-distributed-cluster-state daemon-conf)
:storm-cluster-state (mk-storm-cluster-state daemon-conf)
:tmp-dirs (atom [nimbus-tmp zk-tmp])
:zookeeper zk-handle
:shared-context context}
supervisor-confs (if (sequential? supervisors)
supervisors
(repeat supervisors {}))]
(doseq [sc supervisor-confs]
(add-supervisor cluster-map :ports ports-per-supervisor :conf sc))
cluster-map
))
(defn get-supervisor [cluster-map supervisor-id]
(let [finder-fn #(= (.get-id %) supervisor-id)]
(find-first finder-fn @(:supervisors cluster-map))
))
(defn kill-supervisor [cluster-map supervisor-id]
(let [finder-fn #(= (.get-id %) supervisor-id)
supervisors @(:supervisors cluster-map)
sup (find-first finder-fn
supervisors)]
;; tmp-dir will be taken care of by shutdown
(reset! (:supervisors cluster-map) (remove-first finder-fn supervisors))
(.shutdown sup)
))
(defn kill-local-storm-cluster [cluster-map]
(.shutdown (:nimbus cluster-map))
(.close (:state cluster-map))
(.disconnect (:storm-cluster-state cluster-map))
(doseq [s @(:supervisors cluster-map)]
(.shutdown-all-workers s)
;; race condition here? will it launch the workers again?
(supervisor/kill-supervisor s))
(psim/kill-all-processes)
(log-message "Shutting down in process zookeeper")
(zk/shutdown-inprocess-zookeeper (:zookeeper cluster-map))
(log-message "Done shutting down in process zookeeper")
(doseq [t @(:tmp-dirs cluster-map)]
(log-message "Deleting temporary path " t)
(rmr t)
))
(defn wait-until-cluster-waiting
"Wait until the cluster is idle. Should be used with time simulation."
[cluster-map]
;; wait until all workers, supervisors, and nimbus is waiting
(let [supervisors @(:supervisors cluster-map)
workers (filter (partial satisfies? common/DaemonCommon) (psim/all-processes))
daemons (concat
[(:nimbus cluster-map)]
supervisors
workers) ; because a worker may already be dead
]
(while (not (every? (memfn waiting?) daemons))
(Thread/sleep 10)
)))
(defn advance-cluster-time
([cluster-map secs increment-secs]
(loop [left secs]
(when (> left 0)
(let [diff (min left increment-secs)]
(advance-time-secs! diff)
(wait-until-cluster-waiting cluster-map)
(recur (- left diff))
))))
([cluster-map secs]
(advance-cluster-time cluster-map secs 1)
))
(defmacro with-local-cluster [[cluster-sym & args] & body]
`(let [~cluster-sym (mk-local-storm-cluster ~@args)]
(try
~@body
(catch Throwable t#
(log-error t# "Error in cluster")
)
(finally
(kill-local-storm-cluster ~cluster-sym)))
))
(defmacro with-simulated-time-local-cluster [& args]
`(with-simulated-time
(with-local-cluster ~@args)))
;; TODO: should take in a port symbol and find available port automatically
(defmacro with-inprocess-zookeeper [port & body]
`(with-local-tmp [tmp#]
(let [zks# (zk/mk-inprocess-zookeeper tmp# ~port)]
(try
~@body
(finally
(zk/shutdown-inprocess-zookeeper zks#)
))
)))
(defn submit-local-topology [nimbus storm-name conf topology]
(.submitTopology nimbus storm-name nil (to-json conf) topology))
(defn submit-mocked-assignment [nimbus storm-name conf topology task->component task->node+port]
(with-var-roots [nimbus/mk-task-component-assignments (fn [& ignored] task->component)
nimbus/compute-new-task->node+port (fn [& ignored] task->node+port)]
(submit-local-topology nimbus storm-name conf topology)
))
(defn mk-capture-launch-fn [capture-atom]
(fn [conf shared-context storm-id supervisor-id port worker-id _]
(let [existing (get @capture-atom [supervisor-id port] [])]
(swap! capture-atom assoc [supervisor-id port] (conj existing storm-id))
)))
(defn find-worker-id [supervisor-conf port]
(let [supervisor-state (supervisor-state supervisor-conf)
worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)]
(first ((reverse-map worker->port) port))
))
(defn find-worker-port [supervisor-conf worker-id]
(let [supervisor-state (supervisor-state supervisor-conf)
worker->port (.get supervisor-state common/LS-APPROVED-WORKERS)
]
(worker->port worker-id)
))
(defn mk-capture-shutdown-fn [capture-atom]
(let [existing-fn supervisor/shutdown-worker]
(fn [conf supervisor-id worker-id worker-thread-pids-atom]
(let [port (find-worker-port conf worker-id)
existing (get @capture-atom [supervisor-id port] 0)]
(swap! capture-atom assoc [supervisor-id port] (inc existing))
(existing-fn conf supervisor-id worker-id worker-thread-pids-atom)
))))
(defmacro capture-changed-workers [& body]
`(let [launch-captured# (atom {})
shutdown-captured# (atom {})]
(with-var-roots [supervisor/launch-worker (mk-capture-launch-fn launch-captured#)
supervisor/shutdown-worker (mk-capture-shutdown-fn shutdown-captured#)]
~@body
{:launched @launch-captured#
:shutdown @shutdown-captured#}
)))
(defmacro capture-launched-workers [& body]
`(:launched (capture-changed-workers ~@body)))
(defmacro capture-shutdown-workers [& body]
`(:shutdown (capture-changed-workers ~@body)))
(defnk aggregated-stat [cluster-map storm-name stat-key :component-ids nil]
(let [state (:storm-cluster-state cluster-map)
storm-id (common/get-storm-id state storm-name)
component->tasks (reverse-map
(common/storm-task-info
state
storm-id))
component->tasks (if component-ids
(select-keys component->tasks component-ids)
component->tasks)
task-ids (apply concat (vals component->tasks))
heartbeats (dofor [id task-ids] (.task-heartbeat state storm-id id))
stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
(reduce + stats)
))
(defn emitted-spout-tuples [cluster-map topology storm-name]
(aggregated-stat cluster-map
storm-name
:emitted
:component-ids (keys (.get_spouts topology))))
(defn transferred-tuples [cluster-map storm-name]
(aggregated-stat cluster-map storm-name :transferred))
(defn acked-tuples [cluster-map storm-name]
(aggregated-stat cluster-map storm-name :acked))
(defn simulate-wait [cluster-map]
(if (Time/isSimulating)
(advance-cluster-time cluster-map 10)
(Thread/sleep 100)
))
;; TODO: mock-sources needs to be able to mock out state spouts as well
(defnk complete-topology [cluster-map topology :mock-sources {} :storm-conf {}]
(let [storm-name (str "topologytest-" (uuid))
state (:storm-cluster-state cluster-map)
spouts (.get_spouts topology)
bolts (.get_bolts topology)
replacements (map-val (fn [v]
(FixedTupleSpout.
(for [tup v]
(if (map? tup)
(FixedTuple. (:stream tup) (:values tup))
tup))))
mock-sources)
all-streams (apply concat
(for [[id spec] (merge (clojurify-structure spouts) (clojurify-structure bolts))]
(for [[stream _] (.. spec get_common get_streams)]
(GlobalStreamId. id stream))))
capturer (TupleCaptureBolt. storm-name)
]
(doseq [[id spout] replacements]
(let [spout-spec (get spouts id)]
(.set_spout_object spout-spec (serialize-component-object spout))
))
(doseq [[_ spout-spec] (clojurify-structure spouts)]
(when-not (instance? FixedTupleSpout (deserialized-component-object (.get_spout_object spout-spec)))
(throw (RuntimeException. "Cannot complete topology unless every spout is a FixedTupleSpout (or mocked to be)"))
))
(.set_bolts topology
(assoc (clojurify-structure bolts)
(uuid)
(Bolt.
(into {} (for [id all-streams] [id (mk-global-grouping)]))
(serialize-component-object capturer)
(mk-plain-component-common {} nil))
))
(submit-local-topology (:nimbus cluster-map) storm-name storm-conf topology)
(let [num-source-tuples (reduce +
(for [[_ spout-spec] spouts]
(-> (.get_spout_object spout-spec)
deserialized-component-object
.getSourceTuples
count)
))
storm-id (common/get-storm-id state storm-name)]
(while (< (+ (FixedTupleSpout/getNumAcked storm-id)
(FixedTupleSpout/getNumFailed storm-id))
num-source-tuples)
(simulate-wait cluster-map))
(.killTopology (:nimbus cluster-map) storm-name)
(while (.assignment-info state storm-id nil)
(simulate-wait cluster-map))
(FixedTupleSpout/clear storm-id))
(.getResults capturer)
))
(defn read-tuples
([results component-id stream-id]
(let [fixed-tuples (get results component-id [])]
(mapcat
(fn [ft]
(if (= stream-id (. ft stream))
[(vec (. ft values))]))
fixed-tuples)
))
([results component-id]
(read-tuples results component-id Utils/DEFAULT_STREAM_ID)
))
(defn ms= [& args]
(apply = (map multi-set args)))
(def TRACKER-BOLT-ID "+++tracker-bolt")
(defn mk-tracked-topology
"Spouts are of form [spout & options], bolts are of form [inputs bolt & options]"
[tracked-cluster spouts-map bolts-map]
(let [track-id (::track-id tracked-cluster)
spouts-map (into {}
(for [[id [spout & options]] spouts-map]
[id
(apply mk-spout-spec
(SpoutTracker. spout track-id)
options)]))
bolts-map (into {}
(for [[id [inputs bolt & options]] bolts-map]
[id
(apply mk-bolt-spec
inputs
(BoltTracker. bolt track-id)
options)]))
]
{:topology (mk-topology spouts-map bolts-map)
:last-spout-emit (atom 0)
:cluster tracked-cluster
}))
(defn assoc-track-id [cluster track-id]
(assoc cluster ::track-id track-id))
(defn increment-global! [id key]
(-> (RegisteredGlobalState/getState id)
(get key)
.incrementAndGet))
(defn global-amt [id key]
(-> (RegisteredGlobalState/getState id)
(get key)
.get
))
(defmacro with-tracked-cluster [[cluster-sym & cluster-args] & body]
`(let [id# (uuid)]
(RegisteredGlobalState/setState id#
(doto (ConcurrentHashMap.)
(.put "spout-emitted" (AtomicInteger. 0))
(.put "transferred" (AtomicInteger. 0))
(.put "processed" (AtomicInteger. 0))))
(with-var-roots [acker/mk-acker-bolt (let [old# acker/mk-acker-bolt]
(fn [& args#]
(BoltTracker. (apply old# args#) id#)
))
worker/mk-transfer-fn (let [old# worker/mk-transfer-fn]
(fn [& args#]
(let [transferrer# (apply old# args#)]
(fn [& transfer-args#]
;; (log-message "Transferring: " transfer-args#)
(increment-global! id# "transferred")
(apply transferrer# transfer-args#)
))))
]
(with-local-cluster [~cluster-sym ~@cluster-args]
(let [~cluster-sym (assoc-track-id ~cluster-sym id#)]
~@body)
))
(RegisteredGlobalState/clearState id#)
))
(defn tracked-wait
"Waits until topology is idle and 'amt' more tuples have been emitted by spouts."
([tracked-topology]
(tracked-wait tracked-topology 1))
([tracked-topology amt]
(let [target (+ amt @(:last-spout-emit tracked-topology))
track-id (-> tracked-topology :cluster ::track-id)
waiting? (fn []
(or (not= target (global-amt track-id "spout-emitted"))
(not= (global-amt track-id "transferred")
(global-amt track-id "processed"))
))]
(while (waiting?)
(Thread/sleep 5))
(reset! (:last-spout-emit tracked-topology) target)
)))
(defn ^{:dirty-hack true} fake-tuple [fields values]
(let [ task->component {1 "1"}
topo (mk-topology
{"1" (mk-spout-spec
(feeder-spout fields))}
{})
context (TopologyContext. topo task->component "fake" "" "" 1)]
(Tuple. context values 1 "default")
))