/
nimbus.clj
2612 lines (2390 loc) · 135 KB
/
nimbus.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
;; Licensed to the Apache Software Foundation (ASF) under one
;; or more contributor license agreements. See the NOTICE file
;; distributed with this work for additional information
;; regarding copyright ownership. The ASF licenses this file
;; to you 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.
(ns org.apache.storm.daemon.nimbus
(:import [org.apache.thrift.server THsHaServer THsHaServer$Args])
(:import [org.apache.storm.generated KeyNotFoundException])
(:import [org.apache.storm.security INimbusCredentialPlugin])
(:import [org.apache.storm.blobstore LocalFsBlobStore])
(:import [org.apache.thrift.protocol TBinaryProtocol TBinaryProtocol$Factory])
(:import [org.apache.thrift.exception])
(:import [org.apache.thrift.transport TNonblockingServerTransport TNonblockingServerSocket])
(:import [org.apache.commons.io FileUtils])
(:import [javax.security.auth Subject])
(:import [org.apache.storm.security.auth NimbusPrincipal])
(:import [java.nio ByteBuffer]
[java.util Collections List HashMap]
[org.apache.storm.generated NimbusSummary])
(:import [java.nio ByteBuffer]
[java.util Collections List HashMap ArrayList Iterator HashSet])
(:import [org.apache.storm.blobstore AtomicOutputStream BlobStoreAclHandler
InputStreamWithMeta KeyFilter KeySequenceNumber BlobSynchronizer BlobStoreUtils])
(:import [java.io File FileOutputStream FileInputStream])
(:import [java.net InetAddress ServerSocket BindException])
(:import [java.nio.channels Channels WritableByteChannel])
(:import [org.apache.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils])
(:use [org.apache.storm.scheduler.DefaultScheduler])
(:import [org.apache.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
(:import [org.apache.storm.nimbus NimbusInfo])
(:import [org.apache.storm.scheduler.resource ResourceUtils])
(:import [org.apache.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils TupleUtils ThriftTopologyUtils
BufferFileInputStream BufferInputStream])
(:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
BeginDownloadResult ListBlobsResult ComponentPageInfo TopologyPageInfo LogConfig LogLevel LogLevelAction
ProfileRequest ProfileAction NodeInfo SupervisorPageInfo WorkerSummary WorkerResources ComponentType])
(:import [org.apache.storm.daemon Shutdownable])
(:import [org.apache.storm.cluster ClusterStateContext DaemonType])
(:use [org.apache.storm util config log timer zookeeper local-state])
(:require [org.apache.storm [cluster :as cluster]
[converter :as converter]
[stats :as stats]])
(:require [org.apache.storm.ui.core :as ui])
(:require [clojure.set :as set])
(:import [org.apache.storm.zookeeper AclEnforcement])
(:import [org.apache.storm.daemon.common StormBase Assignment])
(:use [org.apache.storm.daemon common])
(:use [org.apache.storm config])
(:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
(:import [org.apache.storm.utils VersionInfo Time ConfigUtils]
(org.apache.storm.metric ClusterMetricsConsumerExecutor)
(org.apache.storm.metric.api IClusterMetricsConsumer$ClusterInfo DataPoint IClusterMetricsConsumer$SupervisorInfo)
(org.apache.storm Config)
(com.google.common.collect Sets))
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [metrics.meters :refer [defmeter mark!]])
(:require [metrics.gauges :refer [defgauge]])
(:gen-class
:methods [^{:static true} [launch [org.apache.storm.scheduler.INimbus] void]]))
(defmeter nimbus:num-submitTopologyWithOpts-calls)
(defmeter nimbus:num-submitTopology-calls)
(defmeter nimbus:num-killTopologyWithOpts-calls)
(defmeter nimbus:num-killTopology-calls)
(defmeter nimbus:num-rebalance-calls)
(defmeter nimbus:num-activate-calls)
(defmeter nimbus:num-deactivate-calls)
(defmeter nimbus:num-debug-calls)
(defmeter nimbus:num-setWorkerProfiler-calls)
(defmeter nimbus:num-getComponentPendingProfileActions-calls)
(defmeter nimbus:num-setLogConfig-calls)
(defmeter nimbus:num-uploadNewCredentials-calls)
(defmeter nimbus:num-beginFileUpload-calls)
(defmeter nimbus:num-uploadChunk-calls)
(defmeter nimbus:num-finishFileUpload-calls)
(defmeter nimbus:num-beginFileDownload-calls)
(defmeter nimbus:num-downloadChunk-calls)
(defmeter nimbus:num-getNimbusConf-calls)
(defmeter nimbus:num-getLogConfig-calls)
(defmeter nimbus:num-getTopologyConf-calls)
(defmeter nimbus:num-getTopology-calls)
(defmeter nimbus:num-getUserTopology-calls)
(defmeter nimbus:num-getClusterInfo-calls)
(defmeter nimbus:num-getLeader-calls)
(defmeter nimbus:num-isTopologyNameAllowed-calls)
(defmeter nimbus:num-getTopologyInfoWithOpts-calls)
(defmeter nimbus:num-getTopologyInfo-calls)
(defmeter nimbus:num-getTopologyPageInfo-calls)
(defmeter nimbus:num-getSupervisorPageInfo-calls)
(defmeter nimbus:num-getComponentPageInfo-calls)
(defmeter nimbus:num-shutdown-calls)
(def STORM-VERSION (VersionInfo/getVersion))
(defn file-cache-map [conf]
(TimeCacheMap.
(int (conf NIMBUS-FILE-COPY-EXPIRATION-SECS))
(reify TimeCacheMap$ExpiredCallback
(expire [this id stream]
(.close stream)
))
))
(defn mk-scheduler [conf inimbus]
(let [forced-scheduler (.getForcedScheduler inimbus)
scheduler (cond
forced-scheduler
(do (log-message "Using forced scheduler from INimbus " (class forced-scheduler))
forced-scheduler)
(conf STORM-SCHEDULER)
(do (log-message "Using custom scheduler: " (conf STORM-SCHEDULER))
(-> (conf STORM-SCHEDULER) new-instance))
:else
(do (log-message "Using default scheduler")
(DefaultScheduler.)))]
(.prepare scheduler conf)
scheduler
))
(def NIMBUS-ZK-ACLS ZooDefs$Ids/CREATOR_ALL_ACL)
(defn mk-zk-client [conf]
(let [zk-servers (conf STORM-ZOOKEEPER-SERVERS)
zk-port (conf STORM-ZOOKEEPER-PORT)
zk-root (conf STORM-ZOOKEEPER-ROOT)]
(if (and zk-servers zk-port)
(mk-client conf zk-servers zk-port (if (Utils/isZkAuthenticationConfiguredStormServer conf) NIMBUS-ZK-ACLS nil) :root zk-root
:auth-conf conf))))
(defmulti blob-sync cluster-mode)
(defnk is-leader [nimbus :throw-exception true]
(let [leader-elector (:leader-elector nimbus)]
(if (.isLeader leader-elector) true
(if throw-exception
(let [leader-address (.getLeader leader-elector)]
(throw (RuntimeException. (str "not a leader, current leader is " leader-address))))))))
(defn mk-blob-cache-map
"Constructs a TimeCacheMap instance with a blob store timeout whose
expiration callback invokes cancel on the value held by an expired entry when
that value is an AtomicOutputStream and calls close otherwise."
[conf]
(TimeCacheMap.
(int (conf NIMBUS-BLOBSTORE-EXPIRATION-SECS))
(reify TimeCacheMap$ExpiredCallback
(expire [this id stream]
(if (instance? AtomicOutputStream stream)
(.cancel stream)
(.close stream))))))
(defn mk-bloblist-cache-map
"Constructs a TimeCacheMap instance with a blobstore timeout and no callback
function."
[conf]
(TimeCacheMap. (int (conf NIMBUS-BLOBSTORE-EXPIRATION-SECS))))
(defn create-tology-action-notifier [conf]
(when-not (clojure.string/blank? (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN))
(let [instance (new-instance (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN))]
(try
(.prepare instance conf)
instance
(catch Exception e
(log-warn-error e "Ingoring exception, Could not initialize " (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN)))))))
(defn mk-cluster-metrics-consumer-executors [storm-conf]
(map
(fn [consumer]
(ClusterMetricsConsumerExecutor. (get consumer "class")
(get consumer "argument")))
(get storm-conf STORM-CLUSTER-METRICS-CONSUMER-REGISTER)))
(defn nimbus-data [conf inimbus]
(let [forced-scheduler (.getForcedScheduler inimbus)
blob-store (Utils/getNimbusBlobStore conf (NimbusInfo/fromConf conf))
zk-client (mk-zk-client conf)]
{:conf conf
:nimbus-host-port-info (NimbusInfo/fromConf conf)
:inimbus inimbus
:authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
:impersonation-authorization-handler (mk-authorization-handler (conf NIMBUS-IMPERSONATION-AUTHORIZER) conf)
:submitted-count (atom 0)
:storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
(Utils/isZkAuthenticationConfiguredStormServer
conf)
NIMBUS-ZK-ACLS)
:context (ClusterStateContext. DaemonType/NIMBUS))
:submit-lock (Object.)
:sched-lock (Object.)
:cred-update-lock (Object.)
:log-update-lock (Object.)
:heartbeats-cache (atom {})
:downloaders (file-cache-map conf)
:uploaders (file-cache-map conf)
:blob-store blob-store
:blob-downloaders (mk-blob-cache-map conf)
:blob-uploaders (mk-blob-cache-map conf)
:blob-listers (mk-bloblist-cache-map conf)
:uptime (uptime-computer)
:validator (new-instance (conf NIMBUS-TOPOLOGY-VALIDATOR))
:timer (mk-timer :kill-fn (fn [t]
(log-error t "Error when processing event")
(exit-process! 20 "Error when processing an event")
))
:scheduler (mk-scheduler conf inimbus)
:zk-client zk-client
:leader-elector (zk-leader-elector conf zk-client blob-store)
:id->sched-status (atom {})
:node-id->resources (atom {}) ;;resources of supervisors
:id->resources (atom {}) ;;resources of topologies
:id->worker-resources (atom {}) ; resources of workers per topology
:cred-renewers (AuthUtils/GetCredentialRenewers conf)
:topology-history-lock (Object.)
:topo-history-state (nimbus-topo-history-state conf)
:nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)
:nimbus-topology-action-notifier (create-tology-action-notifier conf)
:cluster-consumer-executors (mk-cluster-metrics-consumer-executors conf)
}))
(defn inbox [nimbus]
(master-inbox (:conf nimbus)))
(defn- get-subject
[]
(let [req (ReqContext/context)]
(.subject req)))
(defn- read-storm-conf [conf storm-id blob-store]
(clojurify-structure
(Utils/fromCompressedJsonConf
(.readBlob blob-store (master-stormconf-key storm-id) (get-subject)))))
(declare delay-event)
(declare mk-assignments)
(defn get-nimbus-subject
[]
(let [subject (Subject.)
principal (NimbusPrincipal.)
principals (.getPrincipals subject)]
(.add principals principal)
subject))
(def nimbus-subject
(get-nimbus-subject))
(defn- get-key-list-from-id
[conf id]
(log-debug "set keys id = " id "set = " #{(master-stormcode-key id) (master-stormjar-key id) (master-stormconf-key id)})
(if (local-mode? conf)
[(master-stormcode-key id) (master-stormconf-key id)]
[(master-stormcode-key id) (master-stormjar-key id) (master-stormconf-key id)]))
(defn kill-transition [nimbus storm-id]
(fn [kill-time]
(let [delay (if kill-time
kill-time
(get (read-storm-conf (:conf nimbus) storm-id (:blob-store nimbus))
TOPOLOGY-MESSAGE-TIMEOUT-SECS))]
(delay-event nimbus
storm-id
delay
:remove)
{
:status {:type :killed}
:topology-action-options {:delay-secs delay :action :kill}})
))
(defn rebalance-transition [nimbus storm-id status]
(fn [time num-workers executor-overrides]
(let [delay (if time
time
(get (read-storm-conf (:conf nimbus) storm-id (:blob-store nimbus))
TOPOLOGY-MESSAGE-TIMEOUT-SECS))]
(delay-event nimbus
storm-id
delay
:do-rebalance)
{:status {:type :rebalancing}
:prev-status status
:topology-action-options (-> {:delay-secs delay :action :rebalance}
(assoc-non-nil :num-workers num-workers)
(assoc-non-nil :component->executors executor-overrides))
})))
(defn do-rebalance [nimbus storm-id status storm-base]
(let [rebalance-options (:topology-action-options storm-base)]
(.update-storm! (:storm-cluster-state nimbus)
storm-id
(-> {}
(assoc-non-nil :component->executors (:component->executors rebalance-options))
(assoc-non-nil :num-workers (:num-workers rebalance-options)))))
(mk-assignments nimbus :scratch-topology-id storm-id))
(defn state-transitions [nimbus storm-id status storm-base]
{:active {:inactivate :inactive
:activate nil
:rebalance (rebalance-transition nimbus storm-id status)
:kill (kill-transition nimbus storm-id)
}
:inactive {:activate :active
:inactivate nil
:rebalance (rebalance-transition nimbus storm-id status)
:kill (kill-transition nimbus storm-id)
}
:killed {:startup (fn [] (delay-event nimbus
storm-id
(-> storm-base
:topology-action-options
:delay-secs)
:remove)
nil)
:kill (kill-transition nimbus storm-id)
:remove (fn []
(log-message "Killing topology: " storm-id)
(.remove-storm! (:storm-cluster-state nimbus)
storm-id)
(when (instance? LocalFsBlobStore (:blob-store nimbus))
(doseq [blob-key (get-key-list-from-id (:conf nimbus) storm-id)]
(.remove-blobstore-key! (:storm-cluster-state nimbus) blob-key)
(.remove-key-version! (:storm-cluster-state nimbus) blob-key)))
nil)
}
:rebalancing {:startup (fn [] (delay-event nimbus
storm-id
(-> storm-base
:topology-action-options
:delay-secs)
:do-rebalance)
nil)
:kill (kill-transition nimbus storm-id)
:do-rebalance (fn []
(do-rebalance nimbus storm-id status storm-base)
{:status {:type (:type (:prev-status storm-base))}
:prev-status :rebalancing
:topology-action-options nil})
}})
(defn transition!
([nimbus storm-id event]
(transition! nimbus storm-id event false))
([nimbus storm-id event error-on-no-transition?]
(is-leader nimbus)
(locking (:submit-lock nimbus)
(let [system-events #{:startup}
[event & event-args] (if (keyword? event) [event] event)
storm-base (-> nimbus :storm-cluster-state (.storm-base storm-id nil))
status (:status storm-base)]
;; handles the case where event was scheduled but topology has been removed
(if-not status
(log-message "Cannot apply event " event " to " storm-id " because topology no longer exists")
(let [get-event (fn [m e]
(if (contains? m e)
(m e)
(let [msg (str "No transition for event: " event
", status: " status,
" storm-id: " storm-id)]
(if error-on-no-transition?
(throw-runtime msg)
(do (when-not (contains? system-events event)
(log-message msg))
nil))
)))
transition (-> (state-transitions nimbus storm-id status storm-base)
(get (:type status))
(get-event event))
transition (if (or (nil? transition)
(keyword? transition))
(fn [] transition)
transition)
storm-base-updates (apply transition event-args)
storm-base-updates (if (keyword? storm-base-updates) ;if it's just a symbol, that just indicates new status.
{:status {:type storm-base-updates}}
storm-base-updates)]
(when storm-base-updates
(.update-storm! (:storm-cluster-state nimbus) storm-id storm-base-updates)))))
)))
(defn transition-name! [nimbus storm-name event & args]
(let [storm-id (get-storm-id (:storm-cluster-state nimbus) storm-name)]
(when-not storm-id
(throw (NotAliveException. storm-name)))
(apply transition! nimbus storm-id event args)))
(defn delay-event [nimbus storm-id delay-secs event]
(log-message "Delaying event " event " for " delay-secs " secs for " storm-id)
(schedule (:timer nimbus)
(or delay-secs 0)
#(try (transition! nimbus storm-id event false)
(catch Exception e (log-error e "Exception while trying transition for " storm-id " and event " event)))))
;; active -> reassign in X secs
;; killed -> wait kill time then shutdown
;; active -> reassign in X secs
;; inactive -> nothing
;; rebalance -> wait X seconds then rebalance
;; swap... (need to handle kill during swap, etc.)
;; event transitions are delayed by timer... anything else that comes through (e.g. a kill) override the transition? or just disable other transitions during the transition?
(defmulti setup-jar cluster-mode)
(defmulti clean-inbox cluster-mode)
;; swapping design
;; -- need 2 ports per worker (swap port and regular port)
;; -- topology that swaps in can use all the existing topologies swap ports, + unused worker slots
;; -- how to define worker resources? port range + number of workers?
;; Monitoring (or by checking when nodes go down or heartbeats aren't received):
;; 1. read assignment
;; 2. see which executors/nodes are up
;; 3. make new assignment to fix any problems
;; 4. if a storm exists but is not taken down fully, ensure that storm takedown is launched (step by step remove executors and finally remove assignments)
(defn- assigned-slots
"Returns a map from node-id to a set of ports"
[storm-cluster-state]
(let [assignments (.assignments storm-cluster-state nil)]
(defaulted
(apply merge-with set/union
(for [a assignments
[_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
{node #{port}}
))
{})
))
;; public for testing
(defn all-supervisor-info
([storm-cluster-state] (all-supervisor-info storm-cluster-state nil))
([storm-cluster-state callback]
(let [supervisor-ids (.supervisors storm-cluster-state callback)]
(into {}
(mapcat
(fn [id]
(if-let [info (.supervisor-info storm-cluster-state id)]
[[id info]]
))
supervisor-ids))
)))
(defn- get-version-for-key [key nimbus-host-port-info zk-client]
(let [version (KeySequenceNumber. key nimbus-host-port-info)]
(.getKeySequenceNumber version zk-client)))
(defn get-key-seq-from-blob-store [blob-store]
(let [key-iter (.listKeys blob-store)]
(iterator-seq key-iter)))
(defn- setup-storm-code [nimbus conf storm-id tmp-jar-location storm-conf topology]
(let [subject (get-subject)
storm-cluster-state (:storm-cluster-state nimbus)
blob-store (:blob-store nimbus)
zk-client (:zk-client nimbus)
jar-key (master-stormjar-key storm-id)
code-key (master-stormcode-key storm-id)
conf-key (master-stormconf-key storm-id)
nimbus-host-port-info (:nimbus-host-port-info nimbus)]
(when tmp-jar-location ;;in local mode there is no jar
(.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
(if (instance? LocalFsBlobStore blob-store)
(.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info zk-client))))
(.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
(if (instance? LocalFsBlobStore blob-store)
(.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-version-for-key conf-key nimbus-host-port-info zk-client)))
(.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
(if (instance? LocalFsBlobStore blob-store)
(.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-version-for-key code-key nimbus-host-port-info zk-client)))))
(defn- read-storm-topology [storm-id blob-store]
(Utils/deserialize
(.readBlob blob-store (master-stormcode-key storm-id) (get-subject)) StormTopology))
(defn get-blob-replication-count
[blob-key nimbus]
(if (:blob-store nimbus)
(-> (:blob-store nimbus)
(.getBlobReplication blob-key nimbus-subject))))
(defn- wait-for-desired-code-replication [nimbus conf storm-id]
(let [min-replication-count (conf TOPOLOGY-MIN-REPLICATION-COUNT)
max-replication-wait-time (conf TOPOLOGY-MAX-REPLICATION-WAIT-TIME-SEC)
current-replication-count-jar (if (not (local-mode? conf))
(atom (get-blob-replication-count (master-stormjar-key storm-id) nimbus))
(atom min-replication-count))
current-replication-count-code (atom (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
current-replication-count-conf (atom (get-blob-replication-count (master-stormconf-key storm-id) nimbus))
total-wait-time (atom 0)]
(if (:blob-store nimbus)
(while (and
(or (> min-replication-count @current-replication-count-jar)
(> min-replication-count @current-replication-count-code)
(> min-replication-count @current-replication-count-conf))
(or (neg? max-replication-wait-time)
(< @total-wait-time max-replication-wait-time)))
(sleep-secs 1)
(log-debug "Checking if I am still the leader")
(is-leader nimbus)
(log-debug "waiting for desired replication to be achieved for storm-id = " storm-id
" min-replication-count = " min-replication-count " max-replication-wait-time = " max-replication-wait-time
(if (not (local-mode? conf))"current-replication-count for jar key = " @current-replication-count-jar)
"current-replication-count for code key = " @current-replication-count-code
"current-replication-count for conf key = " @current-replication-count-conf
" total-wait-time " @total-wait-time)
(swap! total-wait-time inc)
(if (not (local-mode? conf))
(reset! current-replication-count-jar (get-blob-replication-count (master-stormjar-key storm-id) nimbus)))
(reset! current-replication-count-code (get-blob-replication-count (master-stormcode-key storm-id) nimbus))
(reset! current-replication-count-conf (get-blob-replication-count (master-stormconf-key storm-id) nimbus))))
(if (and (<= min-replication-count @current-replication-count-conf)
(<= min-replication-count @current-replication-count-code)
(<= min-replication-count @current-replication-count-jar))
(log-message "desired replication count " min-replication-count " achieved, "
"current-replication-count for conf key = " @current-replication-count-conf ", "
"current-replication-count for code key = " @current-replication-count-code ", "
"current-replication-count for jar key = " @current-replication-count-jar)
(log-message "desired replication count of " min-replication-count " not achieved but we have hit the max wait time "
max-replication-wait-time " so moving on with replication count for conf key = " @current-replication-count-conf
" for code key = " @current-replication-count-code "for jar key = " @current-replication-count-jar))))
(defn- read-storm-topology-as-nimbus [storm-id blob-store]
(Utils/deserialize
(.readBlob blob-store (master-stormcode-key storm-id) nimbus-subject) StormTopology))
(declare compute-executor->component)
(defn read-storm-conf-as-nimbus [storm-id blob-store]
(clojurify-structure
(Utils/fromCompressedJsonConf
(.readBlob blob-store (master-stormconf-key storm-id) nimbus-subject))))
(defn fixup-storm-base
[storm-base topo-conf]
(assoc storm-base
:owner (.get topo-conf TOPOLOGY-SUBMITTER-USER)
:principal (.get topo-conf TOPOLOGY-SUBMITTER-PRINCIPAL)))
(defn read-topology-details [nimbus storm-id]
(let [blob-store (:blob-store nimbus)
storm-base (or
(.storm-base (:storm-cluster-state nimbus) storm-id nil)
(throw (NotAliveException. storm-id)))
topology-conf (read-storm-conf-as-nimbus storm-id blob-store)
storm-base (if (nil? (:principal storm-base))
(let [new-sb (fixup-storm-base storm-base topology-conf)]
(.update-storm! (:storm-cluster-state nimbus) storm-id new-sb)
new-sb)
storm-base)
topology (read-storm-topology-as-nimbus storm-id blob-store)
executor->component (->> (compute-executor->component nimbus storm-id)
(map-key (fn [[start-task end-task]]
(ExecutorDetails. (int start-task) (int end-task)))))]
(TopologyDetails. storm-id
topology-conf
topology
(:num-workers storm-base)
executor->component
(:launch-time-secs storm-base)
(:owner storm-base))))
;; Does not assume that clocks are synchronized. Executor heartbeat is only used so that
;; nimbus knows when it's received a new heartbeat. All timing is done by nimbus and
;; tracked through heartbeat-cache
(defn- update-executor-cache [curr hb timeout]
(let [reported-time (:time-secs hb)
{last-nimbus-time :nimbus-time
last-reported-time :executor-reported-time} curr
reported-time (cond reported-time reported-time
last-reported-time last-reported-time
:else 0)
nimbus-time (if (or (not last-nimbus-time)
(not= last-reported-time reported-time))
(current-time-secs)
last-nimbus-time
)]
{:is-timed-out (and
nimbus-time
(>= (time-delta nimbus-time) timeout))
:nimbus-time nimbus-time
:executor-reported-time reported-time
:heartbeat hb}))
(defn update-heartbeat-cache [cache executor-beats all-executors timeout]
(let [cache (select-keys cache all-executors)]
(into {}
(for [executor all-executors :let [curr (cache executor)]]
[executor
(update-executor-cache curr (get executor-beats executor) timeout)]
))))
(defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
(log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
(let [storm-cluster-state (:storm-cluster-state nimbus)
executor-beats (.executor-beats storm-cluster-state storm-id (:executor->node+port existing-assignment))
cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
executor-beats
all-executors
((:conf nimbus) NIMBUS-TASK-TIMEOUT-SECS))]
(swap! (:heartbeats-cache nimbus) assoc storm-id cache)))
(defn- update-all-heartbeats! [nimbus existing-assignments topology->executors]
"update all the heartbeats for all the topologies's executors"
(doseq [[tid assignment] existing-assignments
:let [all-executors (topology->executors tid)]]
(update-heartbeats! nimbus tid all-executors assignment)))
(defn- alive-executors
[nimbus ^TopologyDetails topology-details all-executors existing-assignment]
(log-debug "Computing alive executors for " (.getId topology-details) "\n"
"Executors: " (pr-str all-executors) "\n"
"Assignment: " (pr-str existing-assignment) "\n"
"Heartbeat cache: " (pr-str (@(:heartbeats-cache nimbus) (.getId topology-details)))
)
;; TODO: need to consider all executors associated with a dead executor (in same slot) dead as well,
;; don't just rely on heartbeat being the same
(let [conf (:conf nimbus)
storm-id (.getId topology-details)
executor-start-times (:executor->start-time-secs existing-assignment)
heartbeats-cache (@(:heartbeats-cache nimbus) storm-id)]
(->> all-executors
(filter (fn [executor]
(let [start-time (get executor-start-times executor)
is-timed-out (-> heartbeats-cache (get executor) :is-timed-out)]
(if (and start-time
(or
(< (time-delta start-time)
(conf NIMBUS-TASK-LAUNCH-SECS))
(not is-timed-out)
))
true
(do
(log-message "Executor " storm-id ":" executor " not alive")
false))
)))
doall)))
(defn- to-executor-id [task-ids]
[(first task-ids) (last task-ids)])
(defn- compute-executors [nimbus storm-id]
(let [conf (:conf nimbus)
blob-store (:blob-store nimbus)
storm-base (.storm-base (:storm-cluster-state nimbus) storm-id nil)
component->executors (:component->executors storm-base)
storm-conf (read-storm-conf-as-nimbus storm-id blob-store)
topology (read-storm-topology-as-nimbus storm-id blob-store)
task->component (storm-task-info topology storm-conf)]
(if (nil? component->executors)
[]
(->> (storm-task-info topology storm-conf)
reverse-map
(map-val sort)
(join-maps component->executors)
(map-val (partial apply partition-fixed))
(mapcat second)
(map to-executor-id)
))))
(defn- compute-executor->component [nimbus storm-id]
(let [conf (:conf nimbus)
blob-store (:blob-store nimbus)
executors (compute-executors nimbus storm-id)
topology (read-storm-topology-as-nimbus storm-id blob-store)
storm-conf (read-storm-conf-as-nimbus storm-id blob-store)
task->component (storm-task-info topology storm-conf)
executor->component (into {} (for [executor executors
:let [start-task (first executor)
component (task->component start-task)]]
{executor component}))]
executor->component))
(defn- compute-topology->executors [nimbus storm-ids]
"compute a topology-id -> executors map"
(into {} (for [tid storm-ids]
{tid (set (compute-executors nimbus tid))})))
(defn- compute-topology->alive-executors [nimbus existing-assignments topologies topology->executors scratch-topology-id]
"compute a topology-id -> alive executors map"
(into {} (for [[tid assignment] existing-assignments
:let [topology-details (.getById topologies tid)
all-executors (topology->executors tid)
alive-executors (if (and scratch-topology-id (= scratch-topology-id tid))
all-executors
(set (alive-executors nimbus topology-details all-executors assignment)))]]
{tid alive-executors})))
(defn- compute-supervisor->dead-ports [nimbus existing-assignments topology->executors topology->alive-executors]
(let [dead-slots (into [] (for [[tid assignment] existing-assignments
:let [all-executors (topology->executors tid)
alive-executors (topology->alive-executors tid)
dead-executors (set/difference all-executors alive-executors)
dead-slots (->> (:executor->node+port assignment)
(filter #(contains? dead-executors (first %)))
vals)]]
dead-slots))
supervisor->dead-ports (->> dead-slots
(apply concat)
(map (fn [[sid port]] {sid #{port}}))
(apply (partial merge-with set/union)))]
(or supervisor->dead-ports {})))
(defn- compute-topology->scheduler-assignment [nimbus existing-assignments topology->alive-executors]
"convert assignment information in zk to SchedulerAssignment, so it can be used by scheduler api."
(into {} (for [[tid assignment] existing-assignments
:let [alive-executors (topology->alive-executors tid)
executor->node+port (:executor->node+port assignment)
worker->resources (:worker->resources assignment)
;; making a map from node+port to WorkerSlot with allocated resources
node+port->slot (into {} (for [[[node port] [mem-on-heap mem-off-heap cpu]] worker->resources]
{[node port]
(WorkerSlot. node port mem-on-heap mem-off-heap cpu)}))
executor->slot (into {} (for [[executor [node port]] executor->node+port]
;; filter out the dead executors
(if (contains? alive-executors executor)
{(ExecutorDetails. (first executor)
(second executor))
(get node+port->slot [node port])}
{})))]]
{tid (SchedulerAssignmentImpl. tid executor->slot)})))
(defn- read-all-supervisor-details
[nimbus supervisor->dead-ports topologies missing-assignment-topologies]
"return a map: {supervisor-id SupervisorDetails}"
(let [storm-cluster-state (:storm-cluster-state nimbus)
supervisor-infos (all-supervisor-info storm-cluster-state)
supervisor-details (for [[id info] supervisor-infos]
(SupervisorDetails. id (:meta info) (:resources-map info)))
;; Note that allSlotsAvailableForScheduling
;; only uses the supervisor-details. The rest of the arguments
;; are there to satisfy the INimbus interface.
all-scheduling-slots (->> (.allSlotsAvailableForScheduling
(:inimbus nimbus)
supervisor-details
topologies
(set missing-assignment-topologies))
(map (fn [s] {(.getNodeId s) #{(.getPort s)}}))
(apply merge-with set/union))]
(into {} (for [[sid supervisor-info] supervisor-infos
:let [hostname (:hostname supervisor-info)
scheduler-meta (:scheduler-meta supervisor-info)
dead-ports (supervisor->dead-ports sid)
;; hide the dead-ports from the all-ports
;; these dead-ports can be reused in next round of assignments
all-ports (-> (get all-scheduling-slots sid)
(set/difference dead-ports)
(as-> ports (map int ports)))
supervisor-details (SupervisorDetails. sid hostname scheduler-meta all-ports (:resources-map supervisor-info))]]
{sid supervisor-details}))))
(defn- compute-topology->executor->node+port [scheduler-assignments]
"convert {topology-id -> SchedulerAssignment} to
{topology-id -> {executor [node port]}}"
(map-val (fn [^SchedulerAssignment assignment]
(->> assignment
.getExecutorToSlot
(#(into {} (for [[^ExecutorDetails executor ^WorkerSlot slot] %]
{[(.getStartTask executor) (.getEndTask executor)]
[(.getNodeId slot) (.getPort slot)]})))))
scheduler-assignments))
;; NEW NOTES
;; only assign to supervisors who are there and haven't timed out
;; need to reassign workers with executors that have timed out (will this make it brittle?)
;; need to read in the topology and storm-conf from disk
;; if no slots available and no slots used by this storm, just skip and do nothing
;; otherwise, package rest of executors into available slots (up to how much it needs)
;; in the future could allocate executors intelligently (so that "close" tasks reside on same machine)
;; TODO: slots that have dead executor should be reused as long as supervisor is active
;; (defn- assigned-slots-from-scheduler-assignments [topology->assignment]
;; (->> topology->assignment
;; vals
;; (map (fn [^SchedulerAssignment a] (.getExecutorToSlot a)))
;; (mapcat vals)
;; (map (fn [^WorkerSlot s] {(.getNodeId s) #{(.getPort s)}}))
;; (apply merge-with set/union)
;; ))
(defn num-used-workers [^SchedulerAssignment scheduler-assignment]
(if scheduler-assignment
(count (.getSlots scheduler-assignment))
0 ))
(defn convert-assignments-to-worker->resources [new-scheduler-assignments]
"convert {topology-id -> SchedulerAssignment} to
{topology-id -> {[node port] [mem-on-heap mem-off-heap cpu]}}
Make sure this can deal with other non-RAS schedulers
later we may further support map-for-any-resources"
(map-val (fn [^SchedulerAssignment assignment]
(->> assignment
.getExecutorToSlot
.values
(#(into {} (for [^WorkerSlot slot %]
{[(.getNodeId slot) (.getPort slot)]
[(.getAllocatedMemOnHeap slot) (.getAllocatedMemOffHeap slot) (.getAllocatedCpu slot)]
})))))
new-scheduler-assignments))
(defn compute-new-topology->executor->node+port [new-scheduler-assignments existing-assignments]
(let [new-topology->executor->node+port (compute-topology->executor->node+port new-scheduler-assignments)]
;; print some useful information.
(doseq [[topology-id executor->node+port] new-topology->executor->node+port
:let [old-executor->node+port (-> topology-id
existing-assignments
:executor->node+port)
reassignment (filter (fn [[executor node+port]]
(and (contains? old-executor->node+port executor)
(not (= node+port (old-executor->node+port executor)))))
executor->node+port)]]
(when-not (empty? reassignment)
(let [new-slots-cnt (count (set (vals executor->node+port)))
reassign-executors (keys reassignment)]
(log-message "Reassigning " topology-id " to " new-slots-cnt " slots")
(log-message "Reassign executors: " (vec reassign-executors)))))
new-topology->executor->node+port))
(defrecord TopologyResources [requested-mem-on-heap requested-mem-off-heap requested-cpu
assigned-mem-on-heap assigned-mem-off-heap assigned-cpu])
;; public so it can be mocked out
(defn compute-new-scheduler-assignments [nimbus existing-assignments topologies scratch-topology-id]
(let [conf (:conf nimbus)
storm-cluster-state (:storm-cluster-state nimbus)
topology->executors (compute-topology->executors nimbus (keys existing-assignments))
;; update the executors heartbeats first.
_ (update-all-heartbeats! nimbus existing-assignments topology->executors)
topology->alive-executors (compute-topology->alive-executors nimbus
existing-assignments
topologies
topology->executors
scratch-topology-id)
supervisor->dead-ports (compute-supervisor->dead-ports nimbus
existing-assignments
topology->executors
topology->alive-executors)
topology->scheduler-assignment (compute-topology->scheduler-assignment nimbus
existing-assignments
topology->alive-executors)
missing-assignment-topologies (->> topologies
.getTopologies
(map (memfn getId))
(filter (fn [t]
(let [alle (get topology->executors t)
alivee (get topology->alive-executors t)]
(or (empty? alle)
(not= alle alivee)
(< (-> topology->scheduler-assignment
(get t)
num-used-workers )
(-> topologies (.getById t) .getNumWorkers)))))))
supervisors (read-all-supervisor-details nimbus
supervisor->dead-ports
topologies
missing-assignment-topologies)
cluster (Cluster. (:inimbus nimbus) supervisors topology->scheduler-assignment conf)]
;; set the status map with existing topology statuses
(.setStatusMap cluster (deref (:id->sched-status nimbus)))
;; call scheduler.schedule to schedule all the topologies
;; the new assignments for all the topologies are in the cluster object.
(.schedule (:scheduler nimbus) topologies cluster)
;;merge with existing statuses
(reset! (:id->sched-status nimbus) (merge (deref (:id->sched-status nimbus)) (.getStatusMap cluster)))
(reset! (:node-id->resources nimbus) (.getSupervisorsResourcesMap cluster))
(if-not (conf SCHEDULER-DISPLAY-RESOURCE)
(.updateAssignedMemoryForTopologyAndSupervisor cluster topologies))
; Remove both of swaps below at first opportunity. This is a hack for non-ras scheduler topology and worker resources.
(swap! (:id->resources nimbus) merge (into {} (map (fn [[k v]] [k (->TopologyResources (nth v 0) (nth v 1) (nth v 2)
(nth v 3) (nth v 4) (nth v 5))])
(.getTopologyResourcesMap cluster))))
; Remove this also at first chance
(swap! (:id->worker-resources nimbus) merge
(into {} (map (fn [[k v]] [k (map-val #(doto (WorkerResources.)
(.set_mem_on_heap (nth % 0))
(.set_mem_off_heap (nth % 1))
(.set_cpu (nth % 2))) v)])
(.getWorkerResourcesMap cluster))))
(.getAssignments cluster)))
(defn get-resources-for-topology [nimbus topo-id]
(or (get @(:id->resources nimbus) topo-id)
(try
(let [storm-cluster-state (:storm-cluster-state nimbus)
topology-details (read-topology-details nimbus topo-id)
assigned-resources (->> (.assignment-info storm-cluster-state topo-id nil)
:worker->resources
(vals)
; Default to [[0 0 0]] if there are no values
(#(or % [[0 0 0]]))
; [[on-heap, off-heap, cpu]] -> [[on-heap], [off-heap], [cpu]]
(apply map vector)
; [[on-heap], [off-heap], [cpu]] -> [on-heap-sum, off-heap-sum, cpu-sum]
(map (partial reduce +)))
worker-resources (->TopologyResources (.getTotalRequestedMemOnHeap topology-details)
(.getTotalRequestedMemOffHeap topology-details)
(.getTotalRequestedCpu topology-details)
(nth assigned-resources 0)
(nth assigned-resources 1)
(nth assigned-resources 2))]
(swap! (:id->resources nimbus) assoc topo-id worker-resources)
worker-resources)
(catch KeyNotFoundException e
; This can happen when a topology is first coming up.
; It's thrown by the blobstore code.
(log-error e "Failed to get topology details")
(->TopologyResources 0 0 0 0 0 0)))))
(defn- get-worker-resources-for-topology [nimbus topo-id]
(or (get @(:id->worker-resources nimbus) topo-id)
(try
(let [storm-cluster-state (:storm-cluster-state nimbus)
assigned-resources (->> (.assignment-info storm-cluster-state topo-id nil)
:worker->resources)
worker-resources (into {} (map #(identity {(WorkerSlot. (first (key %)) (second (key %)))
(doto (WorkerResources.)
(.set_mem_on_heap (nth (val %) 0))
(.set_mem_off_heap (nth (val %) 1))
(.set_cpu (nth (val %) 2)))}) assigned-resources))]
(swap! (:id->worker-resources nimbus) assoc topo-id worker-resources)
worker-resources))))
(defn changed-executors [executor->node+port new-executor->node+port]
(let [executor->node+port (if executor->node+port (sort executor->node+port) nil)
new-executor->node+port (if new-executor->node+port (sort new-executor->node+port) nil)
slot-assigned (reverse-map executor->node+port)
new-slot-assigned (reverse-map new-executor->node+port)
brand-new-slots (map-diff slot-assigned new-slot-assigned)]
(apply concat (vals brand-new-slots))
))
(defn newly-added-slots [existing-assignment new-assignment]
(let [old-slots (-> (:executor->node+port existing-assignment)
vals
set)
new-slots (-> (:executor->node+port new-assignment)
vals
set)]
(set/difference new-slots old-slots)))
(defn basic-supervisor-details-map [storm-cluster-state]
(let [infos (all-supervisor-info storm-cluster-state)]
(->> infos
(map (fn [[id info]]
[id (SupervisorDetails. id (:hostname info) (:scheduler-meta info) nil (:resources-map info))]))
(into {}))))
(defn- to-worker-slot [[node port]]
(WorkerSlot. node port))
(defn- fixup-assignment
[assignment td]
(assoc assignment
:owner (.getTopologySubmitter td)))
;; get existing assignment (just the executor->node+port map) -> default to {}
;; filter out ones which have a executor timeout
;; figure out available slots on cluster. add to that the used valid slots to get total slots. figure out how many executors should be in each slot (e.g., 4, 4, 4, 5)
;; only keep existing slots that satisfy one of those slots. for rest, reassign them across remaining slots
;; edge case for slots with no executor timeout but with supervisor timeout... just treat these as valid slots that can be reassigned to. worst comes to worse the executor will timeout and won't assign here next time around
(defnk mk-assignments [nimbus :scratch-topology-id nil]
(if (is-leader nimbus :throw-exception false)
(let [conf (:conf nimbus)
storm-cluster-state (:storm-cluster-state nimbus)
^INimbus inimbus (:inimbus nimbus)
;; read all the topologies