|
54 | 54 | (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
|
55 | 55 | (:use [org.apache.storm util config log converter])
|
56 | 56 | (:require [org.apache.storm [converter :as converter]])
|
| 57 | + (:require [org.apache.storm.ui.core :as ui]) |
57 | 58 | (:require [clojure.set :as set])
|
58 | 59 | (:import [org.apache.storm.daemon.common StormBase Assignment])
|
59 | 60 | (:import [org.apache.storm.zookeeper Zookeeper])
|
60 | 61 | (:use [org.apache.storm.daemon common])
|
61 | 62 | (:use [org.apache.storm config])
|
62 | 63 | (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
|
| 64 | + (:import [org.apache.storm.metric ClusterMetricsConsumerExecutor] |
| 65 | + [org.apache.storm.metric.api IClusterMetricsConsumer$ClusterInfo DataPoint IClusterMetricsConsumer$SupervisorInfo] |
| 66 | + [org.apache.storm Config]) |
63 | 67 | (:import [org.apache.storm.utils VersionInfo LocalState]
|
64 | 68 | [org.json.simple JSONValue])
|
65 | 69 | (:require [clj-time.core :as time])
|
|
167 | 171 | (catch Exception e
|
168 | 172 | (log-warn-error e "Ingoring exception, Could not initialize " (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN)))))))
|
169 | 173 |
|
| 174 | +(defn mk-cluster-metrics-consumer-executors [storm-conf] |
| 175 | + (map |
| 176 | + (fn [consumer] |
| 177 | + (ClusterMetricsConsumerExecutor. (get consumer "class") |
| 178 | + (get consumer "argument"))) |
| 179 | + (get storm-conf STORM-CLUSTER-METRICS-CONSUMER-REGISTER))) |
| 180 | + |
170 | 181 | (defn nimbus-data [conf inimbus]
|
171 | 182 | (let [forced-scheduler (.getForcedScheduler inimbus)]
|
172 | 183 | {:conf conf
|
|
209 | 220 | :topo-history-state (ConfigUtils/nimbusTopoHistoryState conf)
|
210 | 221 | :nimbus-autocred-plugins (AuthUtils/getNimbusAutoCredPlugins conf)
|
211 | 222 | :nimbus-topology-action-notifier (create-tology-action-notifier conf)
|
| 223 | + :cluster-consumer-executors (mk-cluster-metrics-consumer-executors conf) |
212 | 224 | }))
|
213 | 225 |
|
214 | 226 | (defn inbox [nimbus]
|
|
1358 | 1370 | (and (>= val lower)
|
1359 | 1371 | (<= val upper)))
|
1360 | 1372 |
|
1361 |
| -;TODO: when translating this function, you should replace the map-val with a proper for loop HERE |
1362 |
| -(defserverfn service-handler [conf inimbus] |
1363 |
| - (.prepare inimbus conf (ConfigUtils/masterInimbusDir conf)) |
1364 |
| - (log-message "Starting Nimbus with conf " conf) |
1365 |
| - (let [nimbus (nimbus-data conf inimbus) |
1366 |
| - blob-store (:blob-store nimbus) |
1367 |
| - principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf) |
| 1373 | +(defn extract-cluster-metrics [^ClusterSummary summ] |
| 1374 | + (let [cluster-summ (ui/cluster-summary summ "nimbus")] |
| 1375 | + {:cluster-info (IClusterMetricsConsumer$ClusterInfo. (System/currentTimeMillis)) |
| 1376 | + :data-points (map |
| 1377 | + (fn [[k v]] (DataPoint. k v)) |
| 1378 | + (select-keys cluster-summ ["supervisors" "topologies" "slotsTotal" "slotsUsed" "slotsFree" |
| 1379 | + "executorsTotal" "tasksTotal"]))})) |
| 1380 | +(defn extract-supervisors-metrics [^ClusterSummary summ] |
| 1381 | + (let [sups (.get_supervisors summ) |
| 1382 | + supervisors-summ ((ui/supervisor-summary sups) "supervisors")] |
| 1383 | + (map (fn [supervisor-summ] |
| 1384 | + {:supervisor-info (IClusterMetricsConsumer$SupervisorInfo. |
| 1385 | + (supervisor-summ "host") |
| 1386 | + (supervisor-summ "id") |
| 1387 | + (System/currentTimeMillis)) |
| 1388 | + :data-points (map |
| 1389 | + (fn [[k v]] (DataPoint. k v)) |
| 1390 | + (select-keys supervisor-summ ["slotsTotal" "slotsUsed" "totalMem" "totalCpu" |
| 1391 | + "usedMem" "usedCpu"]))}) |
| 1392 | + supervisors-summ))) |
| 1393 | +
|
| 1394 | +(defn send-cluster-metrics-to-executors [nimbus-service nimbus] |
| 1395 | + (let [cluster-summary (.getClusterInfo nimbus-service) |
| 1396 | + cluster-metrics (extract-cluster-metrics cluster-summary) |
| 1397 | + supervisors-metrics (extract-supervisors-metrics cluster-summary)] |
| 1398 | + (dofor |
| 1399 | + [consumer-executor (:cluster-consumer-executors nimbus)] |
| 1400 | + (do |
| 1401 | + (.handleDataPoints consumer-executor (:cluster-info cluster-metrics) (:data-points cluster-metrics)) |
| 1402 | + (dofor |
| 1403 | + [supervisor-metrics supervisors-metrics] |
| 1404 | + (do |
| 1405 | + (.handleDataPoints consumer-executor (:supervisor-info supervisor-metrics) (:data-points supervisor-metrics)))))))) |
| 1406 | +
|
| 1407 | +(defn mk-reified-nimbus [nimbus conf blob-store] |
| 1408 | + (let [principal-to-local (AuthUtils/GetPrincipalToLocalPlugin conf) |
1368 | 1409 | admin-users (or (.get conf NIMBUS-ADMINS) [])
|
1369 | 1410 | get-common-topo-info
|
1370 | 1411 | (fn [^String storm-id operation]
|
|
1400 | 1441 | (doto (ErrorInfo. (:error e) (:time-secs e))
|
1401 | 1442 | (.set_host (:host e))
|
1402 | 1443 | (.set_port (:port e)))))]
|
1403 |
| - (.prepare ^org.apache.storm.nimbus.ITopologyValidator (:validator nimbus) conf) |
1404 |
| -
|
1405 |
| - ;add to nimbuses |
1406 |
| - (.addNimbusHost (:storm-cluster-state nimbus) (.toHostPortString (:nimbus-host-port-info nimbus)) |
1407 |
| - (NimbusSummary. |
1408 |
| - (.getHost (:nimbus-host-port-info nimbus)) |
1409 |
| - (.getPort (:nimbus-host-port-info nimbus)) |
1410 |
| - (Time/currentTimeSecs) |
1411 |
| - false ;is-leader |
1412 |
| - STORM-VERSION)) |
1413 |
| -
|
1414 |
| - (.addToLeaderLockQueue (:leader-elector nimbus)) |
1415 |
| - (cleanup-corrupt-topologies! nimbus) |
1416 |
| - (when (instance? LocalFsBlobStore blob-store) |
1417 |
| - ;register call back for blob-store |
1418 |
| - (.blobstore (:storm-cluster-state nimbus) (fn [] (blob-sync conf nimbus))) |
1419 |
| - (setup-blobstore nimbus)) |
1420 |
| -
|
1421 |
| - (when (is-leader nimbus :throw-exception false) |
1422 |
| - (doseq [storm-id (.activeStorms (:storm-cluster-state nimbus))] |
1423 |
| - (transition! nimbus storm-id :startup))) |
1424 |
| -
|
1425 |
| - (.scheduleRecurring (:timer nimbus) |
1426 |
| - 0 |
1427 |
| - (conf NIMBUS-MONITOR-FREQ-SECS) |
1428 |
| - (fn [] |
1429 |
| - (when-not (conf ConfigUtils/NIMBUS_DO_NOT_REASSIGN) |
1430 |
| - (locking (:submit-lock nimbus) |
1431 |
| - (mk-assignments nimbus))) |
1432 |
| - (do-cleanup nimbus))) |
1433 |
| - ;; Schedule Nimbus inbox cleaner |
1434 |
| - (.scheduleRecurring (:timer nimbus) |
1435 |
| - 0 |
1436 |
| - (conf NIMBUS-CLEANUP-INBOX-FREQ-SECS) |
1437 |
| - (fn [] (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS)))) |
1438 |
| - ;; Schedule nimbus code sync thread to sync code from other nimbuses. |
1439 |
| - (if (instance? LocalFsBlobStore blob-store) |
1440 |
| - (.scheduleRecurring (:timer nimbus) |
1441 |
| - 0 |
1442 |
| - (conf NIMBUS-CODE-SYNC-FREQ-SECS) |
1443 |
| - (fn [] (blob-sync conf nimbus)))) |
1444 |
| - ;; Schedule topology history cleaner |
1445 |
| - (when-let [interval (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)] |
1446 |
| - (.scheduleRecurring (:timer nimbus) |
1447 |
| - 0 |
1448 |
| - (conf LOGVIEWER-CLEANUP-INTERVAL-SECS) |
1449 |
| - (fn [] (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus)))) |
1450 |
| - (.scheduleRecurring (:timer nimbus) |
1451 |
| - 0 |
1452 |
| - (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS) |
1453 |
| - (fn [] |
1454 |
| - (renew-credentials nimbus))) |
1455 |
| -
|
1456 |
| - (def nimbus:num-supervisors (StormMetricsRegistry/registerGauge "nimbus:num-supervisors" |
1457 |
| - (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))) |
1458 |
| -
|
1459 |
| - (StormMetricsRegistry/startMetricsReporters conf) |
1460 |
| - |
1461 | 1444 | (reify Nimbus$Iface
|
1462 | 1445 | (^void submitTopologyWithOpts
|
1463 | 1446 | [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology
|
|
2195 | 2178 | (isWaiting [this]
|
2196 | 2179 | (.isTimerWaiting (:timer nimbus))))))
|
2197 | 2180 |
|
| 2181 | +;TODO: when translating this function, you should replace the map-val with a proper for loop HERE |
| 2182 | +(defserverfn service-handler [conf inimbus] |
| 2183 | + (.prepare inimbus conf (ConfigUtils/masterInimbusDir conf)) |
| 2184 | + (log-message "Starting Nimbus with conf " conf) |
| 2185 | + (let [nimbus (nimbus-data conf inimbus) |
| 2186 | + blob-store (:blob-store nimbus)] |
| 2187 | + (.prepare ^org.apache.storm.nimbus.ITopologyValidator (:validator nimbus) conf) |
| 2188 | +
|
| 2189 | + ;add to nimbuses |
| 2190 | + (.addNimbusHost (:storm-cluster-state nimbus) (.toHostPortString (:nimbus-host-port-info nimbus)) |
| 2191 | + (NimbusSummary. |
| 2192 | + (.getHost (:nimbus-host-port-info nimbus)) |
| 2193 | + (.getPort (:nimbus-host-port-info nimbus)) |
| 2194 | + (Time/currentTimeSecs) |
| 2195 | + false ;is-leader |
| 2196 | + STORM-VERSION)) |
| 2197 | +
|
| 2198 | + (.addToLeaderLockQueue (:leader-elector nimbus)) |
| 2199 | + (cleanup-corrupt-topologies! nimbus) |
| 2200 | + (when (instance? LocalFsBlobStore blob-store) |
| 2201 | + ;register call back for blob-store |
| 2202 | + (.blobstore (:storm-cluster-state nimbus) (fn [] (blob-sync conf nimbus))) |
| 2203 | + (setup-blobstore nimbus)) |
| 2204 | +
|
| 2205 | + (doseq [consumer (:cluster-consumer-executors nimbus)] |
| 2206 | + (.prepare consumer)) |
| 2207 | +
|
| 2208 | + (when (is-leader nimbus :throw-exception false) |
| 2209 | + (doseq [storm-id (.activeStorms (:storm-cluster-state nimbus))] |
| 2210 | + (transition! nimbus storm-id :startup))) |
| 2211 | +
|
| 2212 | + (.scheduleRecurring (:timer nimbus) |
| 2213 | + 0 |
| 2214 | + (conf NIMBUS-MONITOR-FREQ-SECS) |
| 2215 | + (fn [] |
| 2216 | + (when-not (conf ConfigUtils/NIMBUS_DO_NOT_REASSIGN) |
| 2217 | + (locking (:submit-lock nimbus) |
| 2218 | + (mk-assignments nimbus))) |
| 2219 | + (do-cleanup nimbus))) |
| 2220 | + ;; Schedule Nimbus inbox cleaner |
| 2221 | + (.scheduleRecurring (:timer nimbus) |
| 2222 | + 0 |
| 2223 | + (conf NIMBUS-CLEANUP-INBOX-FREQ-SECS) |
| 2224 | + (fn [] (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS)))) |
| 2225 | + ;; Schedule nimbus code sync thread to sync code from other nimbuses. |
| 2226 | + (if (instance? LocalFsBlobStore blob-store) |
| 2227 | + (.scheduleRecurring (:timer nimbus) |
| 2228 | + 0 |
| 2229 | + (conf NIMBUS-CODE-SYNC-FREQ-SECS) |
| 2230 | + (fn [] (blob-sync conf nimbus)))) |
| 2231 | + ;; Schedule topology history cleaner |
| 2232 | + (when-let [interval (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)] |
| 2233 | + (.scheduleRecurring (:timer nimbus) |
| 2234 | + 0 |
| 2235 | + (conf LOGVIEWER-CLEANUP-INTERVAL-SECS) |
| 2236 | + (fn [] (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus)))) |
| 2237 | + (.scheduleRecurring (:timer nimbus) |
| 2238 | + 0 |
| 2239 | + (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS) |
| 2240 | + (fn [] |
| 2241 | + (renew-credentials nimbus))) |
| 2242 | +
|
| 2243 | + (def nimbus:num-supervisors (StormMetricsRegistry/registerGauge "nimbus:num-supervisors" |
| 2244 | + (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))) |
| 2245 | +
|
| 2246 | + (StormMetricsRegistry/startMetricsReporters conf) |
| 2247 | +
|
| 2248 | + (let [reified-inimbus (mk-reified-nimbus nimbus conf blob-store)] |
| 2249 | + (do |
| 2250 | + (if (:cluster-consumer-executors nimbus) |
| 2251 | + (.scheduleRecurring (:timer nimbus) |
| 2252 | + 0 |
| 2253 | + (conf STORM-CLUSTER-METRICS-CONSUMER-PUBLISH-INTERVAL-SECS) |
| 2254 | + (fn [] |
| 2255 | + (when (is-leader nimbus :throw-exception false) |
| 2256 | + (send-cluster-metrics-to-executors reified-inimbus nimbus)))))) |
| 2257 | + reified-inimbus))) |
| 2258 | +
|
| 2259 | +
|
2198 | 2260 | (defn validate-port-available[conf]
|
2199 | 2261 | (try
|
2200 | 2262 | (let [socket (ServerSocket. (conf NIMBUS-THRIFT-PORT))]
|
|
0 commit comments