-
Notifications
You must be signed in to change notification settings - Fork 5
/
onyx_comps.clj
641 lines (568 loc) · 22.9 KB
/
onyx_comps.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
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
(ns clojask.onyx-comps
(:require [clojure.set :as set]
[clojask.clojask-input :as input]
[clojask.clojask-output :as output]
[clojask.clojask-groupby :as groupby]
[clojask.clojask-aggre :as aggre]
[clojask.clojask-join :as join]
[onyx.api :refer :all]
[clojure.string :as string]
[onyx.test-helper :refer [with-test-env feedback-exception!]]
;; [tech.v3.dataset :as ds]
[clojure.data.csv :as csv]
[clojask.utils :refer [eval-res eval-res-ne filter-check]]
[clojask.join :refer [defn-join]])
(:import [java.io BufferedReader FileReader BufferedWriter FileWriter]
[com.clojask.exception ExecutionException]))
;; sample workflow
;;
;; [[:in :sample-worker1]
;; [:in :sample-worker2]
;; [:sample-worker1 :output]
;; [:sample-worker2 :output]]
(def id (java.util.UUID/randomUUID))
(defn workflow-gen
"Generate workflow for running Onyx"
[num-work]
(def workflow []) ;; initialisation
;; for loop for input edges
(doseq [x (range 1 (+ num-work 1))]
(let [worker-name (keyword (str "sample-worker" x))]
(def workflow (conj workflow [:in worker-name]
))))
;; for loop for output edges
(doseq [x (range 1 (+ num-work 1))]
(let [worker-name (keyword (str "sample-worker" x))]
(def workflow (conj workflow [worker-name :output]
))))
;; (println workflow) ; !!debugging
)
;; (defn sample-worker
;; [segment]
;; ;; (println segment)
;; (:id segment)
;; ;; (update-in segment [:map] (fn [n] (assoc n :first (:id segment))))
;; )
(def dataframe (atom nil))
(defn worker-func-gen
[df exception index]
(reset! dataframe df)
(let [operations (.getDesc (:col-info (deref dataframe)))
types (.getType (:col-info (deref dataframe)))
formats (.getFormatter (:col-info (deref dataframe)))
filters (.getFilters (:row-info df))
indices index]
;; (println indices)
(if exception
(defn worker-func
[seg]
(let [id (:id seg)
data (:d seg)] ;; -1 is very important here!
(if (filter-check filters types data)
{:id id :d (mapv (fn [_] (eval-res data types formats operations _)) indices)}
{:id id})))
(defn worker-func
[seg]
(let [id (:id seg)
data (:d seg)]
(if (filter-check filters types data)
{:id id :d (mapv (fn [_] (eval-res-ne data types formats operations _)) indices)}
{:id id})))))
)
(defn worker-func-gen-format
[df exception index]
(reset! dataframe df)
(let [operations (.getDesc (:col-info (deref dataframe)))
types (.getType (:col-info (deref dataframe)))
formats (.getFormatter (:col-info (deref dataframe)))
filters (.getFilters (:row-info df))
indices index]
;; (println indices)
(if exception
(defn worker-func
[seg]
(let [id (:id seg)
data (:d seg)] ;; -1 is very important here!
(if (filter-check filters types data)
{:id id :d (mapv (fn [_] ((or (get formats _) str) (eval-res data types formats operations _))) indices)}
{:id id})))
(defn worker-func
[seg]
(let [id (:id seg)
data (:d seg)]
(if (filter-check filters types data)
{:id id :d (mapv (fn [_] ((or (get formats _) str) (eval-res-ne data types formats operations _))) indices)}
{:id id}))))))
(defn catalog-gen
"Generate the catalog for running Onyx"
[num-work batch-size]
;; initialisation
(def catalog [])
;; input
(def catalog
(conj catalog
{:onyx/name :in
:onyx/plugin :clojask.clojask-input/input
:onyx/type :input
:onyx/medium :seq
:seq/checkpoint? true
:onyx/batch-size batch-size
:onyx/max-peers 1
:input/doc "Reads segments from a core.async channel"}))
;; for loop for sample workers
(doseq [x (range 1 (+ num-work 1))]
(let [worker-name (keyword (str "sample-worker" x))
worker-function (keyword "clojask.onyx-comps" "worker-func")]
(def catalog
(conj catalog
{:onyx/name worker-name
:onyx/fn worker-function
:onyx/type :function
:onyx/batch-size batch-size
:worker/doc "This is a worker node"}
))))
;; output
(def catalog
(conj catalog
{:onyx/name :output
:onyx/plugin :clojask.clojask-output/output
:onyx/type :output
:onyx/medium :core.async ;; this is maked up
:onyx/max-peers 1
:onyx/batch-size batch-size
:output/doc "Writes segments to the file"}))
;; (println catalog) ;; !! debugging
)
(defn catalog-aggre-gen
"Generate the catalog for running Onyx"
[num-work batch-size]
;; initialisation
(def catalog [])
;; input
(def catalog
(conj catalog
{:onyx/name :in
:onyx/plugin :clojask.clojask-input/input
:onyx/type :input
:onyx/medium :seq
:seq/checkpoint? true
:onyx/batch-size batch-size
:onyx/max-peers 1
:input/doc "Reads segments from a core.async channel"}))
;; for loop for sample workers
(doseq [x (range 1 (+ num-work 1))]
(let [worker-name (keyword (str "sample-worker" x))
worker-function (keyword "clojask.onyx-comps" "worker-func")]
(def catalog
(conj catalog
{:onyx/name worker-name
:onyx/fn worker-function
:onyx/type :function
:onyx/batch-size batch-size
:worker/doc "This is a worker node"}))))
;; output
(def catalog
(conj catalog
{:onyx/name :output
:onyx/plugin :clojask.clojask-aggre/output
:onyx/type :output
:onyx/medium :core.async ;; this is maked up
:onyx/max-peers 1
:onyx/batch-size batch-size
:output/doc "Writes segments to the file"}))
;; (println catalog) ;; !! debugging
)
(defn catalog-groupby-gen
"Generate the catalog for running Onyx"
[num-work batch-size]
;; initialisation
(def catalog [])
;; input
(def catalog
(conj catalog
{:onyx/name :in
:onyx/plugin :clojask.clojask-input/input
:onyx/type :input
:onyx/medium :seq
:seq/checkpoint? true
:onyx/batch-size batch-size
:onyx/max-peers 1
:input/doc "Reads segments from a core.async channel"}))
;; for loop for sample workers
(doseq [x (range 1 (+ num-work 1))]
(let [worker-name (keyword (str "sample-worker" x))
worker-function (keyword "clojask.onyx-comps" "worker-func")]
(def catalog
(conj catalog
{:onyx/name worker-name
:onyx/fn worker-function
:onyx/type :function
:onyx/batch-size batch-size
:worker/doc "This is a worker node"}))))
;; output
(def catalog
(conj catalog
{:onyx/name :output
:onyx/plugin :clojask.clojask-groupby/groupby
:onyx/type :output
:onyx/medium :core.async ;; this is maked up
:onyx/max-peers 1
:onyx/batch-size batch-size
:output/doc "Writes segments to the file"}))
;; (println catalog) ;; !! debugging
)
(defn catalog-join-gen
"Generate the catalog for running Onyx"
[num-work batch-size]
;; initialisation
(def catalog [])
;; input
(def catalog
(conj catalog
{:onyx/name :in
:onyx/plugin :clojask.clojask-input/input
:onyx/type :input
:onyx/medium :seq
:seq/checkpoint? true
:onyx/batch-size batch-size
:onyx/max-peers 1
:input/doc "Reads segments from a core.async channel"}))
;; for loop for sample workers
(doseq [x (range 1 (+ num-work 1))]
(let [worker-name (keyword (str "sample-worker" x))
worker-function (keyword "clojask.onyx-comps" "worker-func")]
(def catalog
(conj catalog
{:onyx/name worker-name
:onyx/fn worker-function
:onyx/type :function
:onyx/batch-size batch-size
:worker/doc "This is a worker node"}))))
;; output
(def catalog
(conj catalog
{:onyx/name :output
:onyx/plugin :clojask.clojask-join/join
:onyx/type :output
:onyx/medium :core.async ;; this is maked up
:onyx/max-peers 1
:onyx/batch-size batch-size
:output/doc "Writes segments to the file"}))
;; (println catalog) ;; !! debugging
)
(defn inject-in-reader [event lifecycle]
(let [path (:buffered-reader/filename lifecycle)
;; tmp (println path)
rdr (if (= path nil) nil (FileReader. path))
;; csv-data (csv/read-csv (BufferedReader. rdr))
]
{:seq/rdr rdr
;; :seq/seq (map zipmap ;; make the first row as headers and the following rows as values in a map structure e.g. {:tic AAPL}
;; (->> (first csv-data) ;; take the first row of the csv-data
;; (cons "clojask-id")
;; (map keyword) ;; make the header be the "key" in the map
;; repeat) ;; repeat the process for all the headers
;; (map cons (iterate inc 1) (rest csv-data)))
;; :seq/filters (:clojask/filters lifecycle)
;; :seq/types (:clojask/types lifecycle)
}))
(defn close-reader [event lifecycle]
(if (not= (:seq/rdr event) nil)
(.close (:seq/rdr event))))
;; (defn inject-out-writer [event lifecycle]
;; (let [wrt (BufferedWriter. (FileWriter. (:buffered-writer/filename lifecycle)))]
;; {:seq/wrt wrt}))
;; (defn close-writer [event lifecycle]
;; (.close (:clojask/wtr event)))
;; (def writer-calls
;; {:lifecycle/before-task-start inject-out-writer
;; :lifecycle/after-task-stop close-writer})
(def in-calls
{:lifecycle/before-task-start inject-in-reader
:lifecycle/after-task-stop close-reader})
(defn lifecycle-gen
[source dist order select]
(def lifecycles
[{:lifecycle/task :in
:buffered-reader/filename (if (fn? source) nil source)
;; :clojask/filters (.getFilters (:row-info (deref dataframe)))
;; :clojask/types (.getType (:col-info (deref dataframe)))
:lifecycle/calls ::in-calls}
{:lifecycle/task :in
:lifecycle/calls :clojask.clojask-input/reader-calls}
{:lifecycle/task :output
:buffered-wtr/filename dist
:order order
:indices select
:lifecycle/calls :clojask.clojask-output/writer-calls}]))
(defn lifecycle-aggre-gen
[source dist]
(def lifecycles
[{:lifecycle/task :in
:buffered-reader/filename (if (fn? source) nil source)
;; :clojask/filters (.getFilters (:row-info (deref dataframe)))
;; :clojask/types (.getType (:col-info (deref dataframe)))
:lifecycle/calls ::in-calls}
{:lifecycle/task :in
:lifecycle/calls :clojask.clojask-input/reader-calls}
{:lifecycle/task :output
:buffered-wtr/filename dist
;; :order order
:lifecycle/calls :clojask.clojask-aggre/writer-calls}]))
(defn lifecycle-groupby-gen
[source dist keys key-index]
(def lifecycles
[{:lifecycle/task :in
:buffered-reader/filename (if (fn? source) nil source)
:lifecycle/calls ::in-calls}
{:lifecycle/task :in
:lifecycle/calls :clojask.clojask-input/reader-calls}
{:lifecycle/task :output
:buffered-wtr/filename dist
;; :clojask/groupby-keys keys
:clojask/key-index key-index
:lifecycle/calls :clojask.clojask-groupby/writer-aggre-calls}]))
(defn lifecycle-join-gen
[source dist a b a-keys b-keys a-roll b-roll join-type]
(def lifecycles
[{:lifecycle/task :in
:buffered-reader/filename (if (fn? source) nil source)
:lifecycle/calls ::in-calls}
{:lifecycle/task :in
:lifecycle/calls :clojask.clojask-input/reader-calls}
{:lifecycle/task :output
:buffered-wtr/filename dist
;; :clojask/a-keys a-keys
;; :clojask/b-keys b-keys
:clojask/a-roll a-roll
:clojask/b-roll b-roll
:clojask/a-map (.getKeyIndex (.col-info a))
:clojask/b-map (.getKeyIndex (.col-info b))
:clojask/join-type join-type
:lifecycle/calls :clojask.clojask-join/writer-join-calls}]))
(def num-workers (atom 1))
(defn rem0?
[event old-segment new-segment all-new-segment]
;; (spit "resources/debug.txt" (str new-segment "\n") :append true)
(= (mod (:id new-segment) (deref num-workers)) 0))
(defn rem1?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 1))
(defn rem2?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 2))
(defn rem3?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 3))
(defn rem4?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 4))
(defn rem5?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 5))
(defn rem6?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 6))
(defn rem7?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 7))
(defn rem8?
[event old-segment new-segment all-new-segment]
(= (mod (:id new-segment) (deref num-workers)) 8))
;; [{:flow/from :in
;; :flow/to [:sample-worker1]
;; :flow/predicate :clojask.onyx-comps/rem0?
;; :flow/doc ""}
;; {:flow/from :in
;; :flow/to [:sample-worker2]
;; :flow/predicate :clojask.onyx-comps/rem1?
;; :flow/doc ""}]
(defn flow-cond-gen
"Generate the flow conditions for running Onyx"
[num-work]
(reset! num-workers num-work)
(def flow-conditions []) ;; initialisation
;; for loop for sample workers
(doseq [x (range 1 (+ num-work 1))]
(let [worker-name (keyword (str "sample-worker" x))
predicate-function (keyword "clojask.onyx-comps" (str "rem" (- x 1) "?"))]
(def flow-conditions
(conj flow-conditions
{:flow/from :in
:flow/to [worker-name]
:flow/predicate predicate-function
:worker/doc "This is a flow condition"}
))))
;; (println flow-conditions) ;; !! debugging
)
(defn config-env
[]
(def env-config
{:zookeeper/address "127.0.0.1:2188"
:zookeeper/server? true
:zookeeper.server/port 2188
:onyx/tenancy-id id
:onyx.log/file ".clojask/clojask.log"})
(def peer-config
{:zookeeper/address "127.0.0.1:2188"
:onyx/tenancy-id id
:onyx.peer/job-scheduler :onyx.job-scheduler/balanced
:onyx.messaging/impl :aeron
:onyx.messaging/peer-port 40200
:onyx.messaging/bind-addr "localhost"
:onyx.log/file ".clojask/clojask.log"})
(def env (onyx.api/start-env env-config))
(def peer-group (onyx.api/start-peer-group peer-config))
(def n-peers (count (set (mapcat identity workflow))))
(def v-peers (onyx.api/start-peers n-peers peer-group)))
(defn shutdown
[]
(doseq [v-peer v-peers]
(onyx.api/shutdown-peer v-peer))
(onyx.api/shutdown-peer-group peer-group)
(onyx.api/shutdown-env env))
(defn start-onyx
"start the onyx cluster with the specification inside dataframe"
[num-work batch-size dataframe dist exception order index melt]
(try
(workflow-gen num-work)
(config-env)
(worker-func-gen-format dataframe exception index) ;;need some work
(catalog-gen num-work batch-size)
(lifecycle-gen (.path dataframe) dist order index)
(flow-cond-gen num-work)
(input/inject-dataframe dataframe)
(output/inject-dataframe dataframe)
(output/inject-melt melt)
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[preparing stage] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(let [submission (onyx.api/submit-job peer-config
{:workflow workflow
:catalog catalog
:lifecycles lifecycles
:flow-conditions flow-conditions
:task-scheduler :onyx.task-scheduler/balanced})
job-id (:job-id submission)]
;; (println submission)
(assert job-id "Job was not successfully submitted")
(feedback-exception! peer-config job-id))
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[submit-to-onyx stage] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(shutdown)
(catch Exception e (throw (ExecutionException. (format "[terminate-node stage] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e))))))
"success")
(defn start-onyx-aggre-only
"start the onyx cluster with the specification inside dataframe"
[num-work batch-size dataframe dist exception aggre-func index select]
(try
(workflow-gen num-work)
(config-env)
(worker-func-gen dataframe exception index) ;;need some work
(catalog-aggre-gen num-work batch-size)
(lifecycle-aggre-gen (.path dataframe) dist)
(flow-cond-gen num-work)
(input/inject-dataframe dataframe)
(aggre/inject-dataframe dataframe aggre-func select)
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[preparing stage (aggregate)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(let [submission (onyx.api/submit-job peer-config
{:workflow workflow
:catalog catalog
:lifecycles lifecycles
:flow-conditions flow-conditions
:task-scheduler :onyx.task-scheduler/balanced})
job-id (:job-id submission)]
;; (println submission)
(assert job-id "Job was not successfully submitted")
(feedback-exception! peer-config job-id))
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[submit-to-onyx stage (aggregate)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(shutdown)
(catch Exception e (throw (ExecutionException. (format "[terminate-node stage (aggregate)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e))))))
"success")
(defn start-onyx-groupby
"start the onyx cluster with the specification inside dataframe"
[num-work batch-size dataframe dist groupby-keys groupby-index exception]
;; (println groupby-index)
(try
(workflow-gen num-work)
(config-env)
(worker-func-gen dataframe exception (vec (take (count (.getKeyIndex (.col-info dataframe))) (iterate inc 0)))) ;;need some work
(catalog-groupby-gen num-work batch-size)
(lifecycle-groupby-gen (.path dataframe) dist groupby-keys (.getKeyIndex (.col-info dataframe)))
(flow-cond-gen num-work)
(input/inject-dataframe dataframe)
(groupby/inject-dataframe dataframe groupby-keys groupby-index)
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[preparing stage (groupby)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(let [submission (onyx.api/submit-job peer-config
{:workflow workflow
:catalog catalog
:lifecycles lifecycles
:flow-conditions flow-conditions
:task-scheduler :onyx.task-scheduler/balanced})
job-id (:job-id submission)]
;; (println submission)
(assert job-id "Job was not successfully submitted")
(feedback-exception! peer-config job-id))
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[submit-to-onyx stage (groupby)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(shutdown)
(catch Exception e (throw (ExecutionException. (format "[terminate-node stage (groupby)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e))))))
"success")
(defn start-onyx-join ;; to-do
"start the onyx cluster with the specification inside dataframe"
[num-work batch-size dataframe b dist exception a-keys b-keys a-roll b-roll join-type limit a-index b-index b-format write-index]
;; dataframe means a
(try
(workflow-gen num-work)
(config-env)
(worker-func-gen dataframe exception (take (count (.getKeyIndex (:col-info dataframe))) (iterate inc 0))) ;;need some work
(catalog-join-gen num-work batch-size)
(lifecycle-join-gen (.path dataframe) dist dataframe b a-keys b-keys a-roll b-roll join-type)
(flow-cond-gen num-work)
(input/inject-dataframe dataframe)
(join/inject-dataframe dataframe b a-keys b-keys a-index b-index write-index b-format)
(let [limit (or limit (fn [a b] true))]
(defn-join join-type limit))
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[preparing stage (join)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(let [submission (onyx.api/submit-job peer-config
{:workflow workflow
:catalog catalog
:lifecycles lifecycles
:flow-conditions flow-conditions
:task-scheduler :onyx.task-scheduler/balanced})
job-id (:job-id submission)]
;; (println submission)
(assert job-id "Job was not successfully submitted")
(feedback-exception! peer-config job-id))
(catch Exception e (do
(shutdown)
(throw (ExecutionException. (format "[submit-to-onyx stage (join)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e)))))))
(try
(shutdown)
(catch Exception e (throw (ExecutionException. (format "[terminate-node stage (join)] Refer to .clojask/clojask.log for detailed information. (original error: %s)" (.getMessage e))))))
"success")
;; !! debugging
(defn -main
[& args]
;; (catalog-gen 2 10)
;; (workflow-gen 2)
;; (flow-cond-gen 2)
;; (start-onyx 2 10 )
)