forked from ngrunwald/datasplash
-
Notifications
You must be signed in to change notification settings - Fork 0
/
core.clj
2089 lines (1856 loc) · 80.1 KB
/
core.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
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
(ns ^:no-doc datasplash.core
(:require
[charred.api :as charred]
[clj-stacktrace.core :as st]
[clj-time.coerce :as timc]
[clj-time.format :as timf]
[clojure.edn :as edn]
[clojure.java.shell :refer [sh]]
[clojure.math.combinatorics :as combo]
[clojure.tools.logging :as log]
[superstring.core :as str]
;; to make aot work
[taoensso.nippy :as nippy])
(:import
(clojure.lang MapEntry ExceptionInfo)
(datasplash.fns
ClojureDoFn ClojureStatefulDoFn ClojureCombineFn ClojurePTransform ClojureCustomCoder)
(datasplash.pipelines PipelineWithOptions)
(java.io InputStream OutputStream DataInputStream DataOutputStream StringWriter)
(org.apache.beam.sdk Pipeline)
(org.apache.beam.sdk.coders StringUtf8Coder KvCoder)
(org.apache.beam.sdk.io TextIO FileIO TextIO Compression)
(org.apache.beam.sdk.io.fs EmptyMatchTreatment)
(org.apache.beam.sdk.options PipelineOptionsFactory)
(org.apache.beam.sdk.transforms
Contextful DoFn DoFn$ProcessContext ParDo Create PTransform
Partition Partition$PartitionFn
SerializableFunction WithKeys GroupByKey Distinct Count
Flatten Combine$CombineFn Combine View View$AsSingleton Sample
Watch$Growth)
(org.apache.beam.sdk.transforms.join KeyedPCollectionTuple CoGroupByKey CoGbkResult)
(org.apache.beam.sdk.transforms.windowing
BoundedWindow Window FixedWindows SlidingWindows Sessions Trigger)
(org.apache.beam.sdk.util UserCodeException)
(org.apache.beam.sdk.values
KV PCollection TupleTag TupleTagList PBegin PCollectionList PInput PCollectionTuple)
(org.joda.time DateTimeUtils DateTimeZone Duration Instant)
(org.joda.time.format DateTimeFormat)))
(def required-ns (atom #{}))
(def fast-charred-write (charred/write-json-fn {:indent-str nil}))
(defn write-json-str [data]
(let [w (StringWriter.)]
(fast-charred-write w data)
(.toString w)))
(defn- ->duration
[time]
(or (and (instance? Duration time) time)
(.toStandardDuration time)))
(defn val->clj
[^KV kv]
(let [v (.getValue kv)]
(if (and
(instance? Iterable v)
(not (instance? java.util.Set v))
(not (instance? java.util.Map v)))
(vec v)
v)))
(defn kv->clj
"Coerce from KV to Clojure MapEntry"
[^KV kv]
(MapEntry. (.getKey kv) (val->clj kv)))
(defmethod print-method KV [kv ^java.io.Writer w]
(.write w (pr-str (kv->clj kv))))
(defn unloaded-ns-from-ex
[e]
(loop [todo (st/parse-exception e)
nss (list)]
(let [{:keys [message trace-elems cause]} todo]
(if message
(if (re-find #"clojure\.lang\.Var\$Unbound|call unbound fn|dynamically bind non-dynamic var|Unbound:|Unable to resolve spec:" message)
(let [[_ missing-ns] (or (re-find #"call unbound fn: #'([^/]+)/" message)
(re-find #"Unbound: #'([^/]+)/" message)
(re-find #"Can't dynamically bind non-dynamic var: ([^/]+)/" message)
(re-find #"Unable to resolve spec: :([^/]+)/" message))
ns-to-add (->> trace-elems
(filter #(:clojure %))
(map :ns)
(concat (list missing-ns)))]
(recur cause (concat ns-to-add nss)))
(recur cause nss))
(->> nss
(remove nil?)
(distinct)
(map symbol))))))
(def get-hostname
^{:doc "Try to guess local hostname"}
(memoize
(fn []
(try
(str/trim-newline (:out (sh "hostname")))
(catch Exception e
(str "unknown-hostname error (" (.getMessage e) ")"))))))
(defmacro try-deref
[at]
`(try (deref ~at) (catch ClassCastException e# (require 'datasplash.core) #{})))
(defmacro unwrap-ex-info
[e]
`(let [c# (.getCause ~e)]
(if (and c# (instance? ~e UserCodeException) (instance? ExceptionInfo c#)) c# ~e)))
(defmacro safe-exec-cfg
"Like [[safe-exec]], but takes a map as first argument containing the name of the ptransform for better error message"
[config & body]
`(let [pt-name# (-> ~config
(get :name)
(some-> (name)))]
(try
~@body
(catch ExceptionInfo e#
(if (or (:name (ex-data e#)) (nil? pt-name#))
(throw e#)
(throw (ExceptionInfo. (.getMessage e#)
(if pt-name# (assoc (ex-data e#) :name pt-name#) (ex-data e#))
(if-let [root# (.getCause e#)] root# e#)))))
(catch UserCodeException e#
(throw e#))
(catch Exception e#
;; if var is unbound, nothing has been required
(let [required-at-start# (try-deref required-ns)]
;; About the use of clojure.lang.RT/REQUIRE_LOCK
;; https://ask.clojure.org/index.php/9893/require-is-not-thread-safe?show=9902#c9902
(locking clojure.lang.RT/REQUIRE_LOCK
(let [already-required# (try-deref required-ns)]
(let [nss# (unloaded-ns-from-ex e#)]
(log/debugf "Caught exception %s at runtime with message -> %s => already initialized : %s / candidates for init : %s"
(type e#) (.getMessage e#) (into #{} already-required#) (into [] nss#))
(if (empty? nss#)
(throw (ex-info "Runtime exception intercepted"
(-> {:hostname (get-hostname)}
(cond-> pt-name# (assoc :name pt-name#))) e#))
(let [missings# nss# ;; (remove already-required# nss#)
missing-at-start?# (not (empty? (remove required-at-start# nss#)))]
(if-not (empty? missings#)
(do
(log/debugf "Requiring missing namespaces at runtime: %s" (into [] missings#))
(doseq [missing# missings#]
(require missing#)
(swap! required-ns conj missing#))
~@body)
(if missing-at-start?#
~@body
(do
(log/fatalf
"Dynamic reloading of namespace failure. Already required: %s Attempted: %s"
(into [] nss#) (into [] already-required#))
(throw (ex-info "Dynamic reloading of namespace seems not to work"
(-> {:ns-from-exception (into [] nss#)
:ns-load-attempted (into [] already-required#)
:hostname (get-hostname)}
(cond-> pt-name# (assoc :name pt-name#)))
e#)))))))))))))))
(defmacro safe-exec
"Executes body while trying to sanely require missing ns if the runtime is not yet properly loaded for Clojure in distributed mode. Always wrap try block with this if you intend to eat every Exception produced.
```
(ds/map (fn [elt]
(try
(ds/safe-exec (dangerous-parse-fn elt))
(catch Exception e
(log/error e \"parsing error\"))))
pcoll)
```"
[& body]
`(safe-exec-cfg {} ~@body))
(defn make-kv
{:doc "Returns a KV object from the given arg(s), either [k v] or a MapEntry or seq of two elements."
:added "0.1.0"}
([k v]
(KV/of k v))
([kv] (make-kv (first kv) (second kv))))
(defn dkey
{:doc "Returns the key part of a KV or MapEntry."
:added "0.1.0"}
[elt]
(if (instance? KV elt)
(let [^KV kv elt]
(.getKey kv))
(key elt)))
(defn dval
{:doc "Returns the value part of a KV or MapEntry."
:added "0.1.0"}
[elt]
(if (instance? KV elt)
(val->clj ^KV elt)
(val elt)))
(def ^{:dynamic true :no-doc true} *coerce-to-clj* true)
(def ^{:dynamic true :no-doc true} *context* nil)
(def ^{:dynamic true :no-doc true} *side-inputs* {})
(def ^{:dynamic true :no-doc true} *main-output* nil)
(def ^{:dynamic true :no-doc true} *extra* {})
(def ^{:private true :no-doc true} no-op (constantly nil))
(defn dofn
{:doc "Returns an Instance of DoFn from given Clojure fn"
:added "0.1.0"}
^DoFn
([f {:keys [start-bundle finish-bundle without-coercion-to-clj
side-inputs side-outputs window-fn
stateful? initialize-fn teardown-fn]
:or {start-bundle no-op
finish-bundle no-op
window-fn no-op
teardown-fn no-op}
:as opts}]
(let [process-ctx-fn (fn [^DoFn$ProcessContext context, ^java.util.Map extra]
(safe-exec-cfg
opts
(let [side-ins (persistent!
(reduce
(fn [acc [k pview]]
(assoc! acc k (.sideInput context pview)))
(transient {}) side-inputs))]
(binding [*context* context
*coerce-to-clj* (not without-coercion-to-clj)
*side-inputs* side-ins
*main-output* (when side-outputs (first (sort side-outputs)))
*extra* (persistent!
(reduce
(fn [acc [k v]]
(assoc! acc (keyword k) v))
(transient {}) extra))]
(f context)))))
args {"dofn" process-ctx-fn
"window-fn" window-fn
"start-bundle" start-bundle
"finish-bundle" finish-bundle
"initialize-fn" initialize-fn
"teardown-fn" teardown-fn}]
(if stateful?
(ClojureStatefulDoFn. args)
(ClojureDoFn. args))))
([f] (dofn f {})))
(defn context
{:added "0.1.0"
:doc "In the context of a ParDo, contains the corresponding Context object.
See https://cloud.google.com/dataflow/java-sdk/JavaDoc/com/google/cloud/dataflow/sdk/transforms/DoFn.ProcessContext.html"}
[] *context*)
(defn state
{:added "0.7.0"
:doc "In the context of a ParDo, contains the mutable ValueState."}
[] (*extra* :state))
(defn system
{:added "0.7.0"
:doc "In the context of a ParDo, contains the mutable ValueState."}
[] (*extra* :system))
(defn side-inputs
{:doc "In the context of a ParDo, returns the corresponding side inputs as a map from names to values.
Example:
```
(let [input (ds/generate-input [1 2 3 4 5] p)
side-input (ds/view (ds/generate-input [{1 :a 2 :b 3 :c 4 :d 5 :e}] p))
proc (ds/map (fn [x] (get-in (ds/side-inputs) [:mapping x]))
{:side-inputs {:mapping side-input}} input)])
```"
:added "0.1.0"}
[] *side-inputs*)
(defn get-element-from-context
"Get element from context in ParDo while applying relevent Clojure type conversions"
[^DoFn$ProcessContext c]
(let [element (.element c)]
(if *coerce-to-clj*
(if (instance? KV element)
(kv->clj element)
element)
element)))
(defrecord MultiResult [kvs])
(defn side-outputs
"Returns multiple outputs keyed by keyword.
Example:
```
(let [input (ds/generate-input [1 2 3 4 5] p)
;; simple and multi are pcoll with their respective elements)
{:keys [simple multi]} (ds/map (fn [x] (ds/side-outputs :simple x :multi (* x 10)))
{:side-outputs [:simple :multi]} input)])
```"
[& kvs]
(MultiResult. (partition 2 kvs)))
(defrecord TimeStamped [timestamp result])
(defn with-timestamp
"Returns element(s) with the given timestamp as Timestamp. Anything that can be coerced by clj-time can be given as input.
It can be nested inside a `(side-outputs)` or outside (in which case it applies to all results).
Example:
```
(ds/map (fn [e] (ds/with-timestamp (clj-time.core/now) (* 2 e)) pcoll))
```"
[timestamp result]
(->TimeStamped (Instant. (timc/to-long timestamp)) result))
(defn output-value!
[^DoFn$ProcessContext context entity bindings]
(let [{:keys [tag timestamp]} bindings]
(cond
(and tag timestamp) (if (= (name tag) (name *main-output*))
(.outputWithTimestamp context entity timestamp)
(.outputWithTimestamp context (TupleTag. (name tag)) entity timestamp))
tag (if (= (name tag) (name *main-output*))
(.output context entity)
(.output context (TupleTag. (name tag)) entity))
timestamp (.outputWithTimestamp context entity timestamp)
:else (.output context entity))))
(defn output-to-context
([tx context result]
(loop [todo [{:entity result :bindings {}}]]
(when-let [{:keys [entity bindings]} (first todo)]
(cond
(instance? MultiResult entity) (recur
(concat (rest todo)
(map (fn [[tag sub-entity]]
{:entity sub-entity
:bindings (assoc bindings :tag tag)})
(:kvs entity))))
(instance? TimeStamped entity) (recur
(conj (rest todo)
{:entity (:result entity)
:bindings (assoc bindings :timestamp (:timestamp entity))}))
:else (do
(output-value! context (tx entity) bindings)
(recur (rest todo)))))))
([context result]
(output-to-context identity context result)))
(defn clj->kv
"Coerce from Clojure data to KV objects"
^KV
[obj]
(cond
(instance? KV obj) obj
(and (sequential? obj) (= 2 (count obj))) (KV/of (first obj) (second obj))
:else (throw (ex-info "Cannot coerce given object to KV"
{:hostname (get-hostname)
:input-object obj
:input-object-type (type obj)}))))
(defn map-fn
"Returns a function that corresponds to a Clojure map operation inside a ParDo"
[f]
(fn [^DoFn$ProcessContext c]
(let [elt (get-element-from-context c)
result (f elt)]
(output-to-context c result))))
(defn map-kv-fn
"Returns a function that corresponds to a Clojure map operation inside a ParDo coercing to KV the return"
[f]
(fn [^DoFn$ProcessContext c]
(let [elt (get-element-from-context c)
result (f elt)]
(output-to-context clj->kv c result))))
(defn mapcat-fn
"Returns a function that corresponds to a Clojure mapcat operation inside a ParDo"
[f]
(fn [^DoFn$ProcessContext c]
(let [elt (get-element-from-context c)
result (f elt)]
(doseq [res result]
(output-to-context c res)))))
(defn pardo-fn
"Returns a function that uses the raw ProcessContext from ParDo"
[f]
(fn [^DoFn$ProcessContext c]
(f c)))
(defn filter-fn
"Returns a function that corresponds to a Clojure filter operation inside a ParDo"
[f]
(fn [^DoFn$ProcessContext c]
(let [elt (get-element-from-context c)
result (f elt)]
(when result
(.output c (.element c))))))
(defn didentity
{:doc "Identity function for use in a ParDo"
:added "0.1.0"}
[^DoFn$ProcessContext c]
(.output c (.element c)))
(alter-var-root #'nippy/*thaw-serializable-allowlist*
(fn [_] (into nippy/default-thaw-serializable-allowlist
#{"org.apache.beam.sdk.values.KV"
"com.google.datastore.v1.Entity"})))
(defn make-nippy-coder
{:doc "Returns an instance of a CustomCoder using nippy for serialization"
:added "0.1.0"}
[]
(let [encode-fn (fn [obj ^OutputStream out]
(safe-exec
(let [dos (DataOutputStream. out)]
(nippy/freeze-to-out! dos obj))))
decode-fn (fn [^InputStream in]
(safe-exec
(let [dis (DataInputStream. in)]
(nippy/thaw-from-in! dis))))]
(ClojureCustomCoder. {"decode-fn" decode-fn "encode-fn" encode-fn})))
(defn make-kv-coder
{:doc "Returns an instance of a KvCoder using by default nippy for serialization."
:added "0.1.0"}
([k-coder v-coder]
(KvCoder/of k-coder v-coder))
([] (make-kv-coder (make-nippy-coder) (make-nippy-coder))))
(defn with-opts
[schema opts ^PTransform ptransform]
(reduce
(fn [tr [k specs]]
(if-let [v (get opts k)]
(if-let [action (get specs :action)]
(action tr v)
tr)
tr))
ptransform schema))
(definterface IApply
(apply [nam ptrans])
(apply [ptrans]))
(defrecord GroupSpecs [specs]
PInput
(expand [this] (into {} (map-indexed (fn [idx x] [(TupleTag. (str idx)) (first x)]) specs)))
(getPipeline [this] (let [^PInput pval (-> specs (first) (first))]
(.getPipeline pval)))
IApply
(apply [this nam ptrans] (Pipeline/applyTransform (name nam) this ptrans))
(apply [this ptrans] (Pipeline/applyTransform this ptrans)))
(defn tapply
[pcoll nam tr]
(if (and nam (seq nam))
(.apply pcoll nam tr)
(.apply pcoll tr)))
(defn pcolltuple->map
[^PCollectionTuple pcolltuple]
(let [all (.getAll pcolltuple)]
(persistent!
(reduce
(fn [acc [^TupleTag tag pcoll]]
(assoc! acc (keyword (.getId tag)) pcoll))
(transient {}) all))))
(declare write-edn-file)
(defn apply-transform
"apply the PTransform to the given Pcoll applying options according to schema."
[pcoll ^PTransform transform schema
{:keys [coder coll-name side-outputs checkpoint] :as options}]
(let [nam (some-> options (:name) (name))
clean-opts (dissoc options :name :coder :coll-name)
configured-transform (with-opts schema clean-opts transform)
bound (tapply pcoll nam configured-transform)
rcoll (if-not side-outputs
(-> bound
(cond-> coder (.setCoder coder))
(cond-> coll-name (.setName coll-name)))
(let [pct (pcolltuple->map bound)]
(if coder
(do
(doseq [^PCollection pcoll (vals pct)]
(.setCoder pcoll coder))
pct)
pct)))]
(when checkpoint
(write-edn-file checkpoint rcoll))
rcoll))
(defn with-opts-docstr
[doc-string & schemas]
(apply str doc-string "\n\nAvailable options:\n\n"
(->> (for [schema schemas
[k {:keys [docstr enum default]}] schema]
(-> (str " - " k " => " docstr)
(cond-> enum (str " | One of " (if (map? enum) (keys enum) enum)))
(cond-> default (str " | Defaults to " default))
(str "\n")))
(distinct)
(sort))))
(defn select-enum-option-fn
[option-name enum-map action]
(fn [transform kw]
(let [enum (get enum-map (keyword kw))]
(if enum
(action transform enum)
(throw
(ex-info (format "%s must be one of %s, %s given"
option-name (keys enum-map) kw)
{:expected (keys enum-map)
:given kw}))))))
(defn select-enum-option-fn-set
[options enum-map action]
(fn [transform options-list]
(let [enums (-> (select-keys enum-map options-list) vals set)]
(if (seq enums)
(action transform enums)
(throw
(ex-info (format "%s must be list of at least one of %s, %s given"
options (keys enum-map) options-list)
{:expected [(keys enum-map)]
:given options-list}))))))
(def base-schema
{:coder {:docstr "Uses a specific Coder for the results of this transform. Usually defaults to some form of nippy-coder."}
:checkpoint {:docstr "Given a path, will store the resulting pcoll at this path in edn to facilitate dev/debug."}})
(def named-schema
(merge
base-schema
{:name {:docstr "Adds a name to the Transform."
:action (fn [transform ^String n] (.named transform n))}}))
(def pardo-schema
(merge
named-schema
{:side-inputs {:docstr "Adds a map of PCollectionViews as side inputs to the underlying ParDo Transform. They can be accessed there by key in the return of side-inputs fn."
:action (fn [transform inputs]
(.withSideInputs transform (map val (sort-by key inputs))))}
:side-outputs {:docstr "Defines as a seq of keywords the output tags for the underlying ParDo Transform. The map fn should return a map with keys set to the same set of keywords."
:action (fn [transform kws]
(let [ordered (sort kws)]
(.withOutputTags transform
(TupleTag. (name (first ordered)))
(TupleTagList/of (map (comp #(TupleTag. %) name)
(rest ordered))))))}
:without-coercion-to-clj {:docstr "Avoids coercing Dataflow types to Clojure, like KV. Coercion will happen by default"}
:initialize-fn {:doc-str "Function of 0 arguments called at worker init. It shoud return an initialized state that can be retrieved at runtime with [[system]] function."
:added "0.7.0"}}))
(defn map-op
[transform {:keys [isomorph? kv?] :as base-options}]
(fn make-map-op
([f {:keys [key-coder value-coder coder] :as options}
^PCollection pcoll]
(let [default-coder (cond
isomorph? (.getCoder pcoll)
kv? (or coder
(KvCoder/of
(or key-coder (make-nippy-coder))
(or value-coder (make-nippy-coder))))
:else (make-nippy-coder))
opts (merge (assoc base-options :coder default-coder) options)
^DoFn bare-dofn (dofn (transform f) opts)
pardo (ParDo/of bare-dofn)]
(apply-transform pcoll pardo pardo-schema opts)))
([f pcoll] (make-map-op f {} pcoll))))
(def
^{:arglists [['f 'pcoll] ['f 'options 'pcoll]]
:added "0.2.0"
:doc
(with-opts-docstr
"Uses a raw pardo-fn as a pardo transform
Function f should be a function of one argument, the Pardo$Context object."
pardo-schema)}
pardo (map-op pardo-fn {:label :pardo}))
(def
^{:arglists [['f 'pcoll] ['f 'options 'pcoll]]
:added "0.1.0"
:doc
(with-opts-docstr
"Returns a PCollection of f applied to every item in the source PCollection.
Function f should be a function of one argument.
Example:
```
(ds/map inc foo)
(ds/map (fn [x] (* x x)) foo)
```
Note: Unlike clojure.core/map, datasplash.api/map takes only one PCollection."
pardo-schema)}
dmap (map-op map-fn {:label :map}))
(def
^{:arglists [['f 'pcoll] ['f 'options 'pcoll]]
:added "0.1.0"
:doc
(with-opts-docstr
"Returns a KV PCollection of f applied to every item in the source PCollection.
Function f should be a function of one argument and return seq of keys/values.
Example:
```
(ds/map-kv (fn [{:keys [month revenue]}] [month revenue]) foo)
```
Note: Unlike clojure.core/map, datasplash.api/map-kv takes only one PCollection."
pardo-schema)}
map-kv (map-op map-kv-fn {:label :map-kv :kv? true}))
(def
^{:arglists [['f 'pcoll] ['f 'options 'pcoll]]
:added "0.1.0"
:doc (with-opts-docstr
"Returns the result of applying concat, or flattening, the result of applying
f to each item in the PCollection. Thus f should return a Clojure or Java collection.
Example:
```
(ds/mapcat (fn [x] [(dec x) x (inc x)]) foo)
```"
pardo-schema)}
dmapcat (map-op mapcat-fn {:label :mapcat}))
(def
^{:arglists [['pred 'pcoll] ['f 'options 'pcoll]]
:added "0.1.0"
:doc (with-opts-docstr
"Returns a PCollection that only contains the items for which (pred item)
returns true.
Example:
```
(ds/filter even? foo)
(ds/filter (fn [x] (even? (* x x))) foo)
```"
pardo-schema)}
dfilter (map-op filter-fn {:label :filter :isomorph? true}))
(defn generate-input
{:doc (with-opts-docstr
"Generates a pcollection from the given collection.
Also accepts empty collections.
See https://beam.apache.org/documentation/transforms/java/other/create/
Example:
```
(ds/generate-input (range 0 1000) pipeline)
```"
base-schema)
:added "0.1.0"}
([coll options ^Pipeline p]
(let [{:keys [coder] :as opts} (merge {:coder (make-nippy-coder)}
(assoc options :label :generate-input))
ptrans (if (empty? coll)
(Create/empty coder)
(Create/of coll))]
(apply-transform p ptrans base-schema opts)))
([coll p] (generate-input coll {} p))
([p] (generate-input [] {} p)))
(definterface ICombineFn
(getReduceFn [])
(getExtractFn [])
(getMergeFn [])
(getInitFn []))
(defn combine-fn
{:doc "Returns a CombineFn instance from given args. See https://beam.apache.org/documentation/programming-guide/#combine
Arguments in order:
- reducef: adds element to accumulator: fn of two arguments, returns updated accumulator
```
(fn [acc elt] (assoc acc (ds/key elt) (ds/val elt)))
```
- extractf: fn taking a single accumulator as arg and returning the final result. Defaults to identity
- combinef: fn taking a variable number of accumulators and returning a single merged accumulator. Defaults to using the reduce fn
```
(fn [& accs] (apply merge accs))
```
- initf: fn of 0 args, returns empty accumulator. Defaults to reduce fn with no args
```
(fn [] {})
```
- output-coder: coder for the resulting PCollection. Defaults to nippy-coder
- acc-coder: coder for the accumulator. Defaults to nippy-coder
This function is reminiscent of the reducers api. In has sensible defaults in order to reuse existing functions. For example, this a a perfectly valid combine-fn that sums all numbers in a pcoll:
```
(combine-fn +)
```
"
:added "0.1.0"}
^Combine$CombineFn
([reducef extractf combinef initf output-coder acc-coder]
(let [extractf (or extractf identity)
combinef (or combinef reducef)
initf (or initf reducef)
output-coder (or output-coder (make-nippy-coder))
acc-coder (or acc-coder (make-nippy-coder))
init-fn (fn [] (safe-exec (initf)))
reduce-fn (fn [acc elt] (safe-exec (reducef acc elt)))
combine-fn (fn [accs] (safe-exec (apply combinef accs)))
extract-fn (fn [acc] (safe-exec (extractf acc)))]
(ClojureCombineFn. {"init-fn" init-fn "reduce-fn" reduce-fn "combine-fn" combine-fn
"combine-fn-raw" combinef "extract-fn" extract-fn}
output-coder acc-coder)))
([reducef extractf combinef initf output-coder] (combine-fn reducef extractf combinef initf output-coder nil))
([reducef extractf combinef initf] (combine-fn reducef extractf combinef initf nil))
([reducef extractf combinef] (combine-fn reducef extractf combinef nil))
([reducef extractf] (combine-fn reducef extractf nil))
([reducef]
(if (map? reducef)
(apply combine-fn ((juxt :reduce :extract :combine :init :output-coder :acc-coder) reducef))
(combine-fn reducef nil))))
(def view-schema
(merge
base-schema
{:default {:docstr "Sets a default value for SingletonView"
:action (fn [transform v]
(assert (instance? transform View$AsSingleton) "Default values can only be set for Singleton views")
(if v
(.withDefaultValue transform v)
transform))}
:type {:docstr "Type of View"
:enum [:singleton :iterable :list :map :multi-map]
:default :singleton}}))
(defn view
{:doc (with-opts-docstr
"Produces a View out of a PColl, to be later consumed as a side-input for example.
See https://beam.apache.org/documentation/transforms/java/other/view/"
view-schema)
:added "0.1.0"}
([{:keys [type]
:or {type :singleton}
:as options}
pcoll]
(let [opts (assoc options :label :view :coder nil)
ptrans (case type
:singleton (View/asSingleton)
:iterable (View/asIterable)
:list (View/asList)
:map (View/asMap)
:multi-map (View/asMultimap))]
(apply-transform pcoll ptrans view-schema opts)))
([pcoll] (view {} pcoll)))
(defn- to-edn*
[^DoFn$ProcessContext c]
(let [elt (.element c)
result (pr-str elt)]
(.output c result)))
(def to-edn (partial (map-op identity {:label :to-edn :coder (StringUtf8Coder/of)}) to-edn*))
(def from-edn (partial dmap #(edn/read-string %)))
(defn sfn
"Returns an instance of SerializableFunction equivalent to f."
^SerializableFunction
[f]
(reify
SerializableFunction
(apply [this input]
(safe-exec (f input)))
clojure.lang.IFn
(invoke [this input]
(safe-exec (f input)))))
(defn partition-fn
"Returns a Partition.PartitionFn if possible"
^Partition$PartitionFn
[f]
(if (instance? Partition$PartitionFn f)
f
(reify
Partition$PartitionFn
(partitionFor [this elem num]
(safe-exec (f elem num))))))
(defn dpartition-by
{:doc (with-opts-docstr
"Partitions the content of pcoll according to the PartitionFn.
See https://beam.apache.org/documentation/programming-guide/#partition
The partition function is given two arguments: the current element and the number of partitions."
named-schema)
:added "0.1.0"}
([f num options ^PCollection pcoll]
(let [opts (assoc options :label :partition-by)
ptrans (Partition/of num (partition-fn f))]
(apply-transform pcoll ptrans base-schema opts)))
([f num pcoll] (dpartition-by f num {} pcoll)))
(defn ->combine-fn
"Returns a CombineFn if f is not one already."
[f]
(if (or
(instance? Combine$CombineFn f)
(instance? SerializableFunction f))
f
(combine-fn f)))
(defn djuxt
{:doc "Creates a CombineFn that applies multiple combiners in one go. Produces a vector of combined results.
'sibling fusion' in Dataflow optimizes multiple independant combiners in the same way, but you might find juxt more concise.
Only works with functions created with combine-fn or native clojure functions, and not with native Dataflow CombineFn
Example:
```
(ds/combine (ds/juxt + *) pcoll)
```"
:added "0.1.0"}
[& fns]
(let [cfs (map ->combine-fn fns)]
(combine-fn
(fn [accs elt]
(into []
(map-indexed
(fn [idx acc] (let [f (.getReduceFn (nth cfs idx))]
(f acc elt))) accs)))
(fn [accs]
(into [] (map-indexed
(fn [idx acc] (let [f (.getExtractFn (nth cfs idx))]
(f acc))) accs)))
(fn [& accs]
(into []
(map-indexed
(fn [idx cf] (let [f (.getMergeFn cf)]
(apply f (mapv (fn [acc] (nth acc idx)) accs))))
cfs)))
(fn []
(mapv (fn [cf] (let [f (.getInitFn cf)]
(f))) cfs))
(.getDefaultOutputCoder (first cfs) nil nil)
(.getAccumulatorCoder (first cfs) nil nil))))
(def kv-coder-schema
{:key-coder {:docstr "Coder to be used for encoding keys in the resulting KV PColl."}
:value-coder {:docstr "Coder to be used for encoding values in the resulting KV PColl."}})
(defn with-keys
{:doc (with-opts-docstr
"Returns a PCollection of KV by applying f on each element of the input PCollection and using the return value as the key and the element as the value.
See https://beam.apache.org/documentation/transforms/java/elementwise/withkeys/
Example:
```
(with-keys even? pcoll)
```"
base-schema kv-coder-schema)
:added "0.1.0"}
([f {:keys [key-coder value-coder coder] :as options} ^PCollection pcoll]
(let [opts (assoc options
:coder (or coder
(KvCoder/of
(or key-coder (make-nippy-coder))
(or value-coder (.getCoder pcoll))))
:label :with-keys)
ptrans (WithKeys/of (sfn f))]
(apply-transform pcoll ptrans base-schema opts)))
([f pcoll] (with-keys f {} pcoll)))
(defn group-by-key
{:doc "Takes a KV PCollection as input and returns a KV PCollection as output of K to list of V.
See https://beam.apache.org/documentation/programming-guide/#groupbykey"
:added "0.1.0"}
([options ^PCollection pcoll]
(let [;parent-coder (.getCoder pcoll)
opts (assoc options :label :group-by-keys)]
(apply-transform pcoll (GroupByKey/create) base-schema opts)))
([pcoll] (group-by-key {} pcoll)))
(defmacro ptransform
{:doc "Generates a PTransform with the given name, apply signature and body. Should rarely by used in user code, see [[pt->>]] for the more general use case in application code.
Example (actual implementation of the group-by transform):
```
(ptransform
:group-by
[^PCollection pcoll]
(->> pcoll
(ds/with-keys f opts)
(ds/group-by-key opts)))
```"
:added "0.1.0"}
[_ input & body]
`(let [body-fn# (fn [~(last input)] ~@body)]
(ClojurePTransform. body-fn#)))
(defmacro pt->>
{:doc "Creates and applies a single named PTransform from a sequence of transforms on a single PCollection. You can use it as you would use ->> in Clojure.
Example:
```
(ds/pt->> :transform-name input-pcollection
(ds/map inc {:name :inc})
(ds/filter even? {:name :even?}))
```"
:added "0.2.0"}
[nam input & body]
`(let [ptrans# (ptransform
~nam
[pcoll#]
(->> pcoll#
~@body))]
(apply-transform ~input ptrans# base-schema {:name ~nam})))
(defmacro pt-cond->>
{:doc "Creates and applies a single named PTransform from a sequence of transforms on a single PCollection according to the results of the given predicates. You can use it as you would use cond->> in Clojure.
Example:
```
(ds/cond->> :transform-name input-pcollection
(:do-inc? config) (ds/map inc {:name :inc})
(:do-filter? config) (ds/filter even? {:name :even?}))
```"
:added "0.2.3"}
[nam input & body]
`(let [ptrans# (ptransform
~nam
[pcoll#]
(cond->> pcoll#
~@body))]
(apply-transform ~input ptrans# base-schema {:name ~nam})))
(defn- group-by-transform
[f options]
(let [safe-opts (dissoc options :name)]
(ptransform
:group-by
[^PCollection pcoll]
(->> pcoll
(with-keys f safe-opts)
(group-by-key safe-opts)))))
(defn dgroup-by
{:doc (with-opts-docstr
"Groups a Pcollection by the result of calling (f item) for each item.
This produces a sequence of KV values, similar to using seq with a
map. Each value will be a list of the values that match key.
Example:
```
(ds/group-by :a foo)
(ds/group-by count foo)
```"
base-schema)
:added "0.1.0"}
([f {:keys [_key-coder _value-coder coder] :as options} ^PCollection pcoll]
(let [opts (-> options
(assoc :coder (or coder nil))
(assoc :label :group-by))
ptrans (group-by-transform f opts)]
(apply-transform pcoll ptrans base-schema opts)))
([f pcoll] (dgroup-by f {} pcoll)))
(defn interface->class
[itf]
(if (instance? Class itf)
itf
(Class/forName (name itf))))
(def ^:dynamic *pipeline-builder-caller* "unknown")
(defn create-timestamp
[]
(let [formatter (-> "MMddHHmmss"
(DateTimeFormat/forPattern)
(.withZone DateTimeZone/UTC))]
(.print formatter (DateTimeUtils/currentTimeMillis))))
(defn job-name-template
[tpl args]
(-> tpl
(str/replace #"%U" (or (System/getProperty "user.name")
(:user (:job args))
"nemo"))