diff --git a/pom.xml b/pom.xml
index 831059aba9b..37dbb199a90 100644
--- a/pom.xml
+++ b/pom.xml
@@ -215,6 +215,7 @@
0.9.0
16.0.1
3.9.0.Final
+ 1.0.2
1.6.6
2.1
1.7.7
@@ -829,6 +830,11 @@
jackson-databind
${jackson.version}
+
+ uk.org.lidalia
+ sysout-over-slf4j
+ ${sysout-over-slf4j.version}
+
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 8de24612119..e86eb1994fb 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -41,6 +41,10 @@
from the classpath, classpathDependencyExcludes, but it didn't work in practice.
This is here as a work around to place it at the beginning of the classpath
even though maven does not officially support ordering of the classpath.-->
+
+ uk.org.lidalia
+ sysout-over-slf4j
+
log4j
log4j
@@ -521,6 +525,7 @@
org.clojure:core.incubator
io.dropwizard.metrics:*
metrics-clojure:*
+ uk.org.lidalia:*
@@ -725,6 +730,10 @@
metrics.utils
org.apache.storm.shade.metrics.utils
+
+ uk.org.lidalia
+ org.apache.storm.shade.uk.org.lidalia
+
diff --git a/storm-core/src/clj/org/apache/storm/LocalCluster.clj b/storm-core/src/clj/org/apache/storm/LocalCluster.clj
index df3c180a5b4..bce2a2ecd68 100644
--- a/storm-core/src/clj/org/apache/storm/LocalCluster.clj
+++ b/storm-core/src/clj/org/apache/storm/LocalCluster.clj
@@ -15,7 +15,7 @@
;; limitations under the License.
(ns org.apache.storm.LocalCluster
- (:use [org.apache.storm testing config util])
+ (:use [org.apache.storm testing config])
(:import [org.apache.storm.utils Utils])
(:import [java.util Map])
(:gen-class
@@ -48,7 +48,7 @@
[this name conf topology]
(submit-local-topology
(:nimbus (. this state)) name conf topology)
- (let [hook (get-configured-class conf STORM-TOPOLOGY-SUBMISSION-NOTIFIER-PLUGIN)]
+ (let [hook (Utils/getConfiguredClass conf STORM-TOPOLOGY-SUBMISSION-NOTIFIER-PLUGIN)]
(when hook (submit-hook hook name conf topology))))
diff --git a/storm-core/src/clj/org/apache/storm/clojure.clj b/storm-core/src/clj/org/apache/storm/clojure.clj
index ff338295cd5..9e1836fb22b 100644
--- a/storm-core/src/clj/org/apache/storm/clojure.clj
+++ b/storm-core/src/clj/org/apache/storm/clojure.clj
@@ -23,7 +23,7 @@
(:import [org.apache.storm.spout SpoutOutputCollector ISpout])
(:import [org.apache.storm.utils Utils])
(:import [org.apache.storm.clojure ClojureBolt ClojureSpout])
- (:import [java.util List])
+ (:import [java.util Collection List])
(:require [org.apache.storm [thrift :as thrift]]))
(defn direct-stream [fields]
@@ -153,6 +153,12 @@
(tuple-values [this collector stream]
this))
+(defn- collectify
+ [obj]
+ (if (or (sequential? obj) (instance? Collection obj))
+ obj
+ [obj]))
+
(defnk emit-bolt! [collector values
:stream Utils/DEFAULT_STREAM_ID :anchor []]
(let [^List anchor (collectify anchor)
diff --git a/storm-core/src/clj/org/apache/storm/cluster.clj b/storm-core/src/clj/org/apache/storm/cluster.clj
index 152423afc0c..9c10775edd2 100644
--- a/storm-core/src/clj/org/apache/storm/cluster.clj
+++ b/storm-core/src/clj/org/apache/storm/cluster.clj
@@ -18,14 +18,16 @@
(:import [org.apache.zookeeper.data Stat ACL Id]
[org.apache.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials NimbusSummary
LogConfig ProfileAction ProfileRequest NodeInfo]
- [java.io Serializable])
+ [java.io Serializable StringWriter PrintWriter]
+ [java.net URLEncoder])
(:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
(:import [org.apache.curator.framework CuratorFramework])
- (:import [org.apache.storm.utils Utils])
+ (:import [org.apache.storm.utils Utils Time])
(:import [org.apache.storm.cluster ClusterState ClusterStateContext ClusterStateListener ConnectionState])
(:import [java.security MessageDigest])
(:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
- (:import [org.apache.storm.nimbus NimbusInfo])
+ (:import [org.apache.storm.nimbus NimbusInfo]
+ [org.apache.storm.zookeeper Zookeeper])
(:use [org.apache.storm util log config converter])
(:require [org.apache.storm [zookeeper :as zk]])
(:require [org.apache.storm.daemon [common :as common]]))
@@ -176,7 +178,7 @@
(defn error-path
[storm-id component-id]
- (str (error-storm-root storm-id) "/" (url-encode component-id)))
+ (str (error-storm-root storm-id) "/" (URLEncoder/encode component-id)))
(def last-error-path-seg "last-error")
@@ -184,7 +186,7 @@
[storm-id component-id]
(str (error-storm-root storm-id)
"/"
- (url-encode component-id)
+ (URLEncoder/encode component-id)
"-"
last-error-path-seg))
@@ -240,6 +242,12 @@
:stats (get executor-stats t)}})))
(into {}))))
+(defn- stringify-error [error]
+ (let [result (StringWriter.)
+ printer (PrintWriter. result)]
+ (.printStackTrace error printer)
+ (.toString result)))
+
;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
(defnk mk-storm-cluster-state
[cluster-state-spec :acls nil :context (ClusterStateContext.)]
@@ -259,7 +267,7 @@
state-id (.register
cluster-state
(fn [type path]
- (let [[subtree & args] (tokenize-path path)]
+ (let [[subtree & args] (Zookeeper/tokenizePath path)]
(condp = subtree
ASSIGNMENTS-ROOT (if (empty? args)
(issue-callback! assignments-callback)
@@ -274,7 +282,8 @@
LOGCONFIG-ROOT (issue-map-callback! log-config-callback (first args))
BACKPRESSURE-ROOT (issue-map-callback! backpressure-callback (first args))
;; this should never happen
- (exit-process! 30 "Unknown callback for subtree " subtree args)))))]
+ (Utils/exitProcess 30 ["Unknown callback for subtree " subtree args])
+ ))))]
(doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE BLOBSTORE-SUBTREE NIMBUSES-SUBTREE
LOGCONFIG-SUBTREE]]
(.mkdirs cluster-state p acls))
@@ -381,7 +390,7 @@
;; long dead worker with a skewed clock overrides all the timestamps. By only checking heartbeats
;; with an assigned node+port, and only reading executors from that heartbeat that are actually assigned,
;; we avoid situations like that
- (let [node+port->executors (reverse-map executor->node+port)
+ (let [node+port->executors (clojurify-structure (Utils/reverseMap executor->node+port))
all-heartbeats (for [[[node port] executors] node+port->executors]
(->> (get-worker-heartbeat this storm-id node port)
(convert-executor-beats executors)
@@ -580,7 +589,7 @@
[this storm-id component-id node port error]
(let [path (error-path storm-id component-id)
last-error-path (last-error-path storm-id component-id)
- data (thriftify-error {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port})
+ data (thriftify-error {:time-secs (Time/currentTimeSecs) :error (stringify-error error) :host node :port port})
_ (.mkdirs cluster-state path acls)
ser-data (Utils/serialize data)
_ (.mkdirs cluster-state path acls)
diff --git a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj b/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
index dcfa8d83257..7253ee07787 100644
--- a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
@@ -16,9 +16,10 @@
(ns org.apache.storm.cluster-state.zookeeper-state-factory
(:import [org.apache.curator.framework.state ConnectionStateListener]
- [org.apache.storm.zookeeper Zookeeper])
+ [org.apache.storm.zookeeper Zookeeper]
+ [org.apache.storm.utils Utils])
(:import [org.apache.zookeeper KeeperException$NoNodeException CreateMode
- Watcher$Event$EventType Watcher$Event$KeeperState]
+ Watcher$Event$EventType Watcher$Event$KeeperState]
[org.apache.storm.cluster ClusterState DaemonType])
(:use [org.apache.storm cluster config log util])
(:require [org.apache.storm [zookeeper :as zk]])
@@ -63,7 +64,7 @@
(register
[this callback]
- (let [id (uuid)]
+ (let [id (Utils/uuid)]
(swap! callbacks assoc id callback)
id))
@@ -73,7 +74,7 @@
(set-ephemeral-node
[this path data acls]
- (Zookeeper/mkdirs zk-writer (parent-path path) acls)
+ (Zookeeper/mkdirs zk-writer (Zookeeper/parentPath path) acls)
(if (Zookeeper/exists zk-writer path false)
(try-cause
(Zookeeper/setData zk-writer path data) ; should verify that it's ephemeral
@@ -92,7 +93,7 @@
(if (Zookeeper/exists zk-writer path false)
(Zookeeper/setData zk-writer path data)
(do
- (Zookeeper/mkdirs zk-writer (parent-path path) acls)
+ (Zookeeper/mkdirs zk-writer (Zookeeper/parentPath path) acls)
(Zookeeper/createNode zk-writer path data CreateMode/PERSISTENT acls))))
(set-worker-hb
diff --git a/storm-core/src/clj/org/apache/storm/command/blobstore.clj b/storm-core/src/clj/org/apache/storm/command/blobstore.clj
index b1496db9aac..924f825a722 100644
--- a/storm-core/src/clj/org/apache/storm/command/blobstore.clj
+++ b/storm-core/src/clj/org/apache/storm/command/blobstore.clj
@@ -17,12 +17,13 @@
(:import [java.io InputStream OutputStream]
[org.apache.storm.generated SettableBlobMeta AccessControl AuthorizationException
KeyNotFoundException]
- [org.apache.storm.blobstore BlobStoreAclHandler])
+ [org.apache.storm.blobstore BlobStoreAclHandler]
+ [org.apache.storm.utils Utils])
(:use [org.apache.storm config]
[clojure.string :only [split]]
[clojure.tools.cli :only [cli]]
[clojure.java.io :only [copy input-stream output-stream]]
- [org.apache.storm blobstore log util])
+ [org.apache.storm blobstore log])
(:gen-class))
(defn update-blob-from-stream
@@ -88,10 +89,10 @@
(defn create-cli [args]
(let [[{file :file acl :acl replication-factor :replication-factor} [key] _] (cli args ["-f" "--file" :default nil]
["-a" "--acl" :default [] :parse-fn as-acl]
- ["-r" "--replication-factor" :default -1 :parse-fn parse-int])
+ ["-r" "--replication-factor" :default -1 :parse-fn #(Integer/parseInt %)])
meta (doto (SettableBlobMeta. acl)
(.set_replication_factor replication-factor))]
- (validate-key-name! key)
+ (Utils/validateKeyName key)
(log-message "Creating " key " with ACL " (pr-str (map access-control-str acl)))
(if file
(with-open [f (input-stream file)]
@@ -140,7 +141,7 @@
(log-message "Current replication factor " blob-replication)
blob-replication)
"--update" (let [[{replication-factor :replication-factor} [key] _]
- (cli new-args ["-r" "--replication-factor" :parse-fn parse-int])]
+ (cli new-args ["-r" "--replication-factor" :parse-fn #(Integer/parseInt %)])]
(if (nil? replication-factor)
(throw (RuntimeException. (str "Please set the replication factor")))
(let [blob-replication (.updateBlobReplication blobstore key replication-factor)]
diff --git a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
index ef9ecbbf375..657e2422ea0 100644
--- a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
@@ -14,6 +14,7 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns org.apache.storm.command.dev-zookeeper
+ (:import [org.apache.storm.utils Utils])
(:use [org.apache.storm zookeeper util config])
(:import [org.apache.storm.utils ConfigUtils])
(:import [org.apache.storm.zookeeper Zookeeper])
@@ -23,6 +24,5 @@
(let [conf (clojurify-structure (ConfigUtils/readStormConfig))
port (conf STORM-ZOOKEEPER-PORT)
localpath (conf DEV-ZOOKEEPER-PATH)]
- (rmr localpath)
- (Zookeeper/mkInprocessZookeeper localpath port)
- ))
+ (Utils/forceDelete localpath)
+ (Zookeeper/mkInprocessZookeeper localpath port)))
diff --git a/storm-core/src/clj/org/apache/storm/command/get_errors.clj b/storm-core/src/clj/org/apache/storm/command/get_errors.clj
index c267390834b..615a5f33cff 100644
--- a/storm-core/src/clj/org/apache/storm/command/get_errors.clj
+++ b/storm-core/src/clj/org/apache/storm/command/get_errors.clj
@@ -21,7 +21,8 @@
[nimbus :as nimbus]
[common :as common]])
(:import [org.apache.storm.generated GetInfoOptions NumErrorsChoice
- TopologySummary ErrorInfo])
+ TopologySummary ErrorInfo]
+ [org.json.simple JSONValue])
(:gen-class))
(defn get-topology-id [name topologies]
@@ -44,9 +45,10 @@
topo-id (get-topology-id name topologies)
topo-info (when (not-nil? topo-id) (.getTopologyInfoWithOpts nimbus topo-id opts))]
(if (or (nil? topo-id) (nil? topo-info))
- (println (to-json {"Failure" (str "No topologies running with name " name)}))
+ (println (JSONValue/toJSONString {"Failure" (str "No topologies running with name " name)}))
(let [topology-name (.get_name topo-info)
topology-errors (.get_errors topo-info)]
- (println (to-json (hash-map
- "Topology Name" topology-name
- "Comp-Errors" (get-component-errors topology-errors)))))))))
+ (println (JSONValue/toJSONString
+ (hash-map
+ "Topology Name" topology-name
+ "Comp-Errors" (get-component-errors topology-errors)))))))))
diff --git a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 8a5eb213d3d..02533386bbe 100644
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@ -15,6 +15,7 @@
;; limitations under the License.
(ns org.apache.storm.command.shell-submission
(:import [org.apache.storm StormSubmitter]
+ [org.apache.storm.utils Utils]
[org.apache.storm.zookeeper Zookeeper])
(:use [org.apache.storm thrift util config log zookeeper])
(:require [clojure.string :as str])
@@ -31,5 +32,4 @@
no-op (.close zk-leader-elector)
jarpath (StormSubmitter/submitJar conf tmpjarpath)
args (concat args [host port jarpath])]
- (exec-command! (str/join " " args))
- ))
+ (Utils/execCommand args)))
diff --git a/storm-core/src/clj/org/apache/storm/config.clj b/storm-core/src/clj/org/apache/storm/config.clj
index 3666e13fb56..e50f0231f40 100644
--- a/storm-core/src/clj/org/apache/storm/config.clj
+++ b/storm-core/src/clj/org/apache/storm/config.clj
@@ -18,7 +18,7 @@
(:import [java.io FileReader File IOException]
[org.apache.storm.generated StormTopology])
(:import [org.apache.storm Config])
- (:import [org.apache.storm.utils Utils LocalState ConfigUtils])
+ (:import [org.apache.storm.utils Utils LocalState ConfigUtils MutableInt])
(:import [org.apache.storm.validation ConfigValidation])
(:import [org.apache.commons.io FileUtils])
(:require [clojure [string :as str]])
@@ -49,6 +49,22 @@
(/ 1)
int))
+(defn- even-sampler
+ [freq]
+ (let [freq (int freq)
+ start (int 0)
+ r (java.util.Random.)
+ curr (MutableInt. -1)
+ target (MutableInt. (.nextInt r freq))]
+ (with-meta
+ (fn []
+ (let [i (.increment curr)]
+ (when (>= i freq)
+ (.set curr start)
+ (.set target (.nextInt r freq))))
+ (= (.get curr) (.get target)))
+ {:rate freq})))
+
;; TODO this function together with sampling-rate are to be replaced with Java version when util.clj is in
(defn mk-stats-sampler
[conf]
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index bb2dc8777e2..5599d28fb9c 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -16,7 +16,8 @@
(ns org.apache.storm.converter
(:import [org.apache.storm.generated SupervisorInfo NodeInfo Assignment WorkerResources
StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
- TopologyActionOptions DebugOptions ProfileRequest])
+ TopologyActionOptions DebugOptions ProfileRequest]
+ [org.apache.storm.utils Utils])
(:use [org.apache.storm util stats log])
(:require [org.apache.storm.daemon [common :as common]]))
@@ -71,6 +72,7 @@
(:worker->resources assignment)))))
thrift-assignment))
+;TODO: when translating this function, you should replace the map-key with a proper for loop HERE
(defn clojurify-executor->node_port [executor->node_port]
(into {}
(map-val
@@ -90,6 +92,7 @@
[(.get_mem_on_heap resources) (.get_mem_off_heap resources) (.get_cpu resources)]])
worker->resources)))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn clojurify-assignment [^Assignment assignment]
(if assignment
(org.apache.storm.daemon.common.Assignment.
@@ -117,12 +120,17 @@
:killed TopologyStatus/KILLED
nil)))
+(defn assoc-non-nil
+ [m k v]
+ (if v (assoc m k v) m))
+
(defn clojurify-rebalance-options [^RebalanceOptions rebalance-options]
(-> {:action :rebalance}
(assoc-non-nil :delay-secs (if (.is_set_wait_secs rebalance-options) (.get_wait_secs rebalance-options)))
(assoc-non-nil :num-workers (if (.is_set_num_workers rebalance-options) (.get_num_workers rebalance-options)))
(assoc-non-nil :component->executors (if (.is_set_num_executors rebalance-options) (into {} (.get_num_executors rebalance-options))))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn thriftify-rebalance-options [rebalance-options]
(if rebalance-options
(let [thrift-rebalance-options (RebalanceOptions.)]
@@ -178,6 +186,7 @@
(.set_enable (get options :enable false))
(.set_samplingpct (get options :samplingpct 10))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn thriftify-storm-base [storm-base]
(doto (StormBase.)
(.set_name (:storm-name storm-base))
@@ -190,6 +199,7 @@
(.set_prev_status (convert-to-status-from-symbol (:prev-status storm-base)))
(.set_component_debug (map-val thriftify-debugoptions (:component->debug storm-base)))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn clojurify-storm-base [^StormBase storm-base]
(if storm-base
(org.apache.storm.daemon.common.StormBase.
@@ -203,6 +213,7 @@
(convert-to-symbol-from-status (.get_prev_status storm-base))
(map-val clojurify-debugoptions (.get_component_debug storm-base)))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn thriftify-stats [stats]
(if stats
(map-val thriftify-executor-stats
@@ -210,6 +221,7 @@
stats))
{}))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn clojurify-stats [stats]
(if stats
(map-val clojurify-executor-stats
diff --git a/storm-core/src/clj/org/apache/storm/daemon/acker.clj b/storm-core/src/clj/org/apache/storm/daemon/acker.clj
index 7c4d6147147..dc05dfcfde2 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/acker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/acker.clj
@@ -14,12 +14,13 @@
;; See the License for the specific language governing permissions and
;; limitations under the License.
(ns org.apache.storm.daemon.acker
- (:import [org.apache.storm.task OutputCollector TopologyContext IBolt])
+ (:import [org.apache.storm.task OutputCollector TopologyContext IBolt]
+ [org.apache.storm.utils Utils])
(:import [org.apache.storm.tuple Tuple Fields])
- (:import [org.apache.storm.utils RotatingMap MutableObject])
+ (:import [org.apache.storm.utils Container RotatingMap MutableObject])
(:import [java.util List Map])
(:import [org.apache.storm Constants])
- (:use [org.apache.storm config util log])
+ (:use [org.apache.storm config log])
(:gen-class
:init init
:implements [org.apache.storm.task.IBolt]
@@ -88,20 +89,20 @@
)))
(defn -init []
- [[] (container)])
+ [[] (Container.)])
-(defn -prepare [this conf context collector]
+(defn -prepare [^org.apache.storm.daemon.acker this conf context collector]
(let [^IBolt ret (mk-acker-bolt)]
- (container-set! (.state ^org.apache.storm.daemon.acker this) ret)
+ (.. this state (set ret))
(.prepare ret conf context collector)
))
-(defn -execute [this tuple]
- (let [^IBolt delegate (container-get (.state ^org.apache.storm.daemon.acker this))]
+(defn -execute [^org.apache.storm.daemon.acker this tuple]
+ (let [^IBolt delegate (.. this state (get))]
(.execute delegate tuple)
))
-(defn -cleanup [this]
- (let [^IBolt delegate (container-get (.state ^org.apache.storm.daemon.acker this))]
+(defn -cleanup [^org.apache.storm.daemon.acker this]
+ (let [^IBolt delegate (.. this state (get))]
(.cleanup delegate)
))
diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj
index d0f8dd9fec6..eb1ec1e5a6c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@ -17,17 +17,17 @@
(:use [org.apache.storm log config util])
(:import [org.apache.storm.generated StormTopology
InvalidTopologyException GlobalStreamId]
- [org.apache.storm.utils ThriftTopologyUtils]
+ [org.apache.storm.utils Utils ConfigUtils IPredicate ThriftTopologyUtils]
[org.apache.storm.daemon.metrics.reporters PreparableReporter]
[com.codahale.metrics MetricRegistry])
- (:import [org.apache.storm.utils Utils ConfigUtils])
(:import [org.apache.storm.daemon.metrics MetricsUtils])
(:import [org.apache.storm.task WorkerTopologyContext])
(:import [org.apache.storm Constants])
(:import [org.apache.storm.metric SystemBolt])
(:import [org.apache.storm.metric EventLoggerBolt])
- (:import [org.apache.storm.security.auth IAuthorizer])
- (:import [java.io InterruptedIOException])
+ (:import [org.apache.storm.security.auth IAuthorizer])
+ (:import [java.io InterruptedIOException]
+ [org.json.simple JSONValue])
(:require [clojure.set :as set])
(:require [org.apache.storm.daemon.acker :as acker])
(:require [org.apache.storm.thrift :as thrift])
@@ -84,10 +84,9 @@
(ExecutorStats. 0 0 0 0 0))
(defn get-storm-id [storm-cluster-state storm-name]
- (let [active-storms (.active-storms storm-cluster-state)]
- (find-first
- #(= storm-name (:storm-name (.storm-base storm-cluster-state % nil)))
- active-storms)
+ (let [active-storms (.active-storms storm-cluster-state)
+ pred (reify IPredicate (test [this x] (= storm-name (:storm-name (.storm-base storm-cluster-state x nil)))))]
+ (Utils/findOne pred active-storms)
))
(defn topology-bases [storm-cluster-state]
@@ -114,12 +113,12 @@
(throw e#))
(catch Throwable t#
(log-error t# "Error on initialization of server " ~(str name))
- (exit-process! 13 "Error on initialization")
+ (Utils/exitProcess 13 "Error on initialization")
)))))
(defn- validate-ids! [^StormTopology topology]
(let [sets (map #(.getFieldValue topology %) thrift/STORM-TOPOLOGY-FIELDS)
- offending (apply any-intersection sets)]
+ offending (apply set/intersection sets)]
(if-not (empty? offending)
(throw (InvalidTopologyException.
(str "Duplicate component ids: " offending))))
@@ -145,9 +144,10 @@
(defn component-conf [component]
(->> component
- .get_common
- .get_json_conf
- from-json))
+ .get_common
+ .get_json_conf
+ (#(if % (JSONValue/parse %)))
+ clojurify-structure))
(defn validate-basic! [^StormTopology topology]
(validate-ids! topology)
@@ -238,7 +238,7 @@
{TOPOLOGY-TICK-TUPLE-FREQ-SECS (storm-conf TOPOLOGY-MESSAGE-TIMEOUT-SECS)})]]
(do
;; this set up tick tuples to cause timeouts to be triggered
- (.set_json_conf common (to-json spout-conf))
+ (.set_json_conf common (JSONValue/toJSONString spout-conf))
(.put_to_streams common ACKER-INIT-STREAM-ID (thrift/output-fields ["id" "init-val" "spout-task"]))
(.put_to_inputs common
(GlobalStreamId. ACKER-COMPONENT-ID ACKER-ACK-STREAM-ID)
@@ -363,6 +363,7 @@
(defn num-start-executors [component]
(thrift/parallelism-hint (.get_common component)))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn storm-task-info
"Returns map from task -> component id"
[^StormTopology user-topology storm-conf]
diff --git a/storm-core/src/clj/org/apache/storm/daemon/drpc.clj b/storm-core/src/clj/org/apache/storm/daemon/drpc.clj
index a07b9efbe56..8e83ca28136 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/drpc.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/drpc.clj
@@ -17,15 +17,18 @@
(ns org.apache.storm.daemon.drpc
(:import [org.apache.storm.security.auth AuthUtils ThriftServer ThriftConnectionType ReqContext])
(:import [org.apache.storm.security.auth.authorizer DRPCAuthorizerBase])
+ (:import [org.apache.storm.utils Utils])
(:import [org.apache.storm.generated DistributedRPC DistributedRPC$Iface DistributedRPC$Processor
DRPCRequest DRPCExecutionException DistributedRPCInvocations DistributedRPCInvocations$Iface
DistributedRPCInvocations$Processor])
(:import [java.util.concurrent Semaphore ConcurrentLinkedQueue
ThreadPoolExecutor ArrayBlockingQueue TimeUnit])
- (:import [org.apache.storm.daemon Shutdownable])
+ (:import [org.apache.storm.daemon Shutdownable]
+ [org.apache.storm.utils Time])
(:import [java.net InetAddress])
(:import [org.apache.storm.generated AuthorizationException]
- [org.apache.storm.utils VersionInfo ConfigUtils])
+ [org.apache.storm.utils VersionInfo ConfigUtils]
+ [org.apache.storm.logging ThriftAccessLogger])
(:use [org.apache.storm config log util])
(:use [org.apache.storm.daemon common])
(:use [org.apache.storm.ui helpers])
@@ -57,7 +60,7 @@
(defn check-authorization
([aclHandler mapping operation context]
(if (not-nil? context)
- (log-thrift-access (.requestID context) (.remoteAddress context) (.principal context) operation))
+ (ThriftAccessLogger/logAccess (.requestID context) (.remoteAddress context) (.principal context) operation))
(if aclHandler
(let [context (or context (ReqContext/context))]
(if-not (.permit aclHandler context operation mapping)
@@ -85,10 +88,10 @@
(swap! id->request dissoc id)
(swap! id->start dissoc id))
my-ip (.getHostAddress (InetAddress/getLocalHost))
- clear-thread (async-loop
+ clear-thread (Utils/asyncLoop
(fn []
(doseq [[id start] @id->start]
- (when (> (time-delta start) (conf DRPC-REQUEST-TIMEOUT-SECS))
+ (when (> (Time/deltaSecs start) (conf DRPC-REQUEST-TIMEOUT-SECS))
(when-let [sem (@id->sem id)]
(.remove (acquire-queue request-queues (@id->function id)) (@id->request id))
(log-warn "Timeout DRPC request id: " id " start at " start)
@@ -107,7 +110,7 @@
^Semaphore sem (Semaphore. 0)
req (DRPCRequest. args id)
^ConcurrentLinkedQueue queue (acquire-queue request-queues function)]
- (swap! id->start assoc id (current-time-secs))
+ (swap! id->start assoc id (Time/currentTimeSecs))
(swap! id->sem assoc id sem)
(swap! id->function assoc id function)
(swap! id->request assoc id req)
@@ -227,9 +230,9 @@
(DistributedRPCInvocations$Processor. drpc-service-handler)
ThriftConnectionType/DRPC_INVOCATIONS)
http-creds-handler (AuthUtils/GetDrpcHttpCredentialsPlugin conf)]
- (add-shutdown-hook-with-force-kill-in-1-sec (fn []
- (if handler-server (.stop handler-server))
- (.stop invoke-server)))
+ (Utils/addShutdownHookWithForceKillIn1Sec (fn []
+ (if handler-server (.stop handler-server))
+ (.stop invoke-server)))
(log-message "Starting Distributed RPC servers...")
(future (.serve invoke-server))
(when (> drpc-http-port 0)
@@ -270,5 +273,5 @@
(.serve handler-server)))))
(defn -main []
- (setup-default-uncaught-exception-handler)
+ (Utils/setupDefaultUncaughtExceptionHandler)
(launch-server!))
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index ab0c8aab524..e2380b74ce5 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -36,7 +36,9 @@
(:import [org.apache.storm Config Constants])
(:import [org.apache.storm.cluster ClusterStateContext DaemonType])
(:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
- (:import [java.util.concurrent ConcurrentLinkedQueue])
+ (:import [java.lang Thread Thread$UncaughtExceptionHandler]
+ [java.util.concurrent ConcurrentLinkedQueue]
+ [org.json.simple JSONValue])
(:require [org.apache.storm [thrift :as thrift]
[cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
(:require [org.apache.storm.daemon [task :as task]])
@@ -109,6 +111,7 @@
:direct
)))
+;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
(defn- outbound-groupings
[^WorkerTopologyContext worker-context this-component-id stream-id out-fields component->grouping topo-conf]
(->> component->grouping
@@ -151,7 +154,7 @@
bolts (.get_bolts topology)]
(cond (contains? spouts component-id) :spout
(contains? bolts component-id) :bolt
- :else (throw-runtime "Could not find " component-id " in topology " topology))))
+ :else (throw (RuntimeException. (str "Could not find " component-id " in topology " topology))))))
(defn executor-selector [executor-data & _] (:type executor-data))
@@ -181,7 +184,8 @@
spec-conf (-> general-context
(.getComponentCommon component-id)
.get_json_conf
- from-json)]
+ (#(if % (JSONValue/parse %)))
+ clojurify-structure)]
(merge storm-conf (apply dissoc spec-conf to-remove))
))
@@ -195,20 +199,20 @@
(let [storm-conf (:storm-conf executor)
error-interval-secs (storm-conf TOPOLOGY-ERROR-THROTTLE-INTERVAL-SECS)
max-per-interval (storm-conf TOPOLOGY-MAX-ERROR-REPORT-PER-INTERVAL)
- interval-start-time (atom (current-time-secs))
+ interval-start-time (atom (Time/currentTimeSecs))
interval-errors (atom 0)
]
(fn [error]
(log-error error)
- (when (> (time-delta @interval-start-time)
+ (when (> (Time/deltaSecs @interval-start-time)
error-interval-secs)
(reset! interval-errors 0)
- (reset! interval-start-time (current-time-secs)))
+ (reset! interval-start-time (Time/currentTimeSecs)))
(swap! interval-errors inc)
(when (<= @interval-errors max-per-interval)
(cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
- (hostname storm-conf)
+ (Utils/hostname storm-conf)
(.getThisWorkerPort (:worker-context executor)) error)
))))
@@ -262,13 +266,15 @@
:task->component (:task->component worker)
:stream->component->grouper (outbound-components worker-context component-id storm-conf)
:report-error (throttled-report-error-fn <>)
- :report-error-and-die (fn [error]
- ((:report-error <>) error)
- (if (or
- (exception-cause? InterruptedException error)
- (exception-cause? java.io.InterruptedIOException error))
- (log-message "Got interrupted excpetion shutting thread down...")
- ((:suicide-fn <>))))
+ :report-error-and-die (reify
+ Thread$UncaughtExceptionHandler
+ (uncaughtException [this _ error]
+ ((:report-error <>) error)
+ (if (or
+ (Utils/exceptionCauseIsInstanceOf InterruptedException error)
+ (Utils/exceptionCauseIsInstanceOf java.io.InterruptedIOException error))
+ (log-message "Got interrupted excpetion shutting thread down...")
+ ((:suicide-fn <>)))))
:sampler (mk-stats-sampler storm-conf)
:backpressure (atom false)
:spout-throttling-metrics (if (= executor-type :spout)
@@ -308,7 +314,7 @@
(when batch-end?
(worker-transfer-fn serializer alist)
(.setObject cached-emit (ArrayList.)))))
- :kill-fn (:report-error-and-die executor-data))))
+ :uncaught-exception-handler (:report-error-and-die executor-data))))
(defn setup-metrics! [executor-data]
(let [{:keys [storm-conf receive-queue worker-context interval->task->metric-registry]} executor-data
@@ -329,7 +335,7 @@
task-id (:task-id task-data)
name->imetric (-> interval->task->metric-registry (get interval) (get task-id))
task-info (IMetricsConsumer$TaskInfo.
- (hostname (:storm-conf executor-data))
+ (Utils/hostname (:storm-conf executor-data))
(.getThisWorkerPort worker-context)
(:component-id executor-data)
task-id
@@ -386,8 +392,9 @@
;; doesn't block (because it's a single threaded queue and the caching/consumer started
;; trick isn't thread-safe)
system-threads [(start-batch-transfer->worker-handler! worker executor-data)]
- handlers (with-error-reaction report-error-and-die
- (mk-threads executor-data task-datas initial-credentials))
+ handlers (try
+ (mk-threads executor-data task-datas initial-credentials)
+ (catch Throwable t (.uncaughtException report-error-and-die nil t)))
threads (concat handlers system-threads)]
(setup-ticks! worker executor-data)
@@ -472,7 +479,7 @@
(if p (* p num-tasks))))
(defn init-spout-wait-strategy [storm-conf]
- (let [ret (-> storm-conf (get TOPOLOGY-SPOUT-WAIT-STRATEGY) new-instance)]
+ (let [ret (-> storm-conf (get TOPOLOGY-SPOUT-WAIT-STRATEGY) Utils/newInstance)]
(.prepare ret storm-conf)
ret
))
@@ -491,6 +498,10 @@
EVENTLOGGER-STREAM-ID
[component-id message-id (System/currentTimeMillis) values]))))
+(defn- bit-xor-vals
+ [vals]
+ (reduce bit-xor 0 vals))
+
(defmethod mk-threads :spout [executor-data task-datas initial-credentials]
(let [{:keys [storm-conf component-id worker-context transfer-fn report-error sampler open-or-prepare-was-called?]} executor-data
^ISpoutWaitStrategy spout-wait-strategy (init-spout-wait-strategy storm-conf)
@@ -506,7 +517,7 @@
2 ;; microoptimize for performance of .size method
(reify RotatingMap$ExpiredCallback
(expire [this id [task-id spout-id tuple-info start-time-ms]]
- (let [time-delta (if start-time-ms (time-delta-ms start-time-ms))]
+ (let [time-delta (if start-time-ms (Time/deltaMs start-time-ms))]
(fail-spout-msg executor-data (get task-datas task-id) spout-id tuple-info time-delta "TIMEOUT" id)
))))
tuple-action-fn (fn [task-id ^TupleImpl tuple]
@@ -523,8 +534,8 @@
[stored-task-id spout-id tuple-finished-info start-time-ms] (.remove pending id)]
(when spout-id
(when-not (= stored-task-id task-id)
- (throw-runtime "Fatal error, mismatched task ids: " task-id " " stored-task-id))
- (let [time-delta (if start-time-ms (time-delta-ms start-time-ms))]
+ (throw (RuntimeException. (str "Fatal error, mismatched task ids: " task-id " " stored-task-id))))
+ (let [time-delta (if start-time-ms (Time/deltaMs start-time-ms))]
(condp = stream-id
ACKER-ACK-STREAM-ID (ack-spout-msg executor-data (get task-datas task-id)
spout-id tuple-finished-info time-delta id)
@@ -538,142 +549,140 @@
has-ackers? (has-ackers? storm-conf)
has-eventloggers? (has-eventloggers? storm-conf)
emitted-count (MutableLong. 0)
- empty-emit-streak (MutableLong. 0)]
-
- [(async-loop
- (fn []
- ;; If topology was started in inactive state, don't call (.open spout) until it's activated first.
- (while (not @(:storm-active-atom executor-data))
- (Thread/sleep 100))
-
- (log-message "Opening spout " component-id ":" (keys task-datas))
- (builtin-metrics/register-spout-throttling-metrics (:spout-throttling-metrics executor-data) storm-conf (:user-context (first (vals task-datas))))
- (doseq [[task-id task-data] task-datas
- :let [^ISpout spout-obj (:object task-data)
- tasks-fn (:tasks-fn task-data)
- send-spout-msg (fn [out-stream-id values message-id out-task-id]
- (.increment emitted-count)
- (let [out-tasks (if out-task-id
- (tasks-fn out-task-id out-stream-id values)
- (tasks-fn out-stream-id values))
- rooted? (and message-id has-ackers?)
- root-id (if rooted? (MessageId/generateId rand))
- ^List out-ids (fast-list-for [t out-tasks] (if rooted? (MessageId/generateId rand)))]
- (fast-list-iter [out-task out-tasks id out-ids]
- (let [tuple-id (if rooted?
- (MessageId/makeRootId root-id id)
- (MessageId/makeUnanchored))
- out-tuple (TupleImpl. worker-context
- values
- task-id
- out-stream-id
- tuple-id)]
- (transfer-fn out-task out-tuple)))
- (if has-eventloggers?
- (send-to-eventlogger executor-data task-data values component-id message-id rand))
- (if (and rooted?
- (not (.isEmpty out-ids)))
- (do
- (.put pending root-id [task-id
- message-id
- {:stream out-stream-id
- :values (if debug? values nil)}
- (if (sampler) (System/currentTimeMillis))])
- (task/send-unanchored task-data
- ACKER-INIT-STREAM-ID
- [root-id (bit-xor-vals out-ids) task-id]))
- (when message-id
- (ack-spout-msg executor-data task-data message-id
- {:stream out-stream-id :values values}
- (if (sampler) 0) "0:")))
- (or out-tasks [])
- ))]]
- (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf (:user-context task-data))
- (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
- :receive receive-queue}
- storm-conf (:user-context task-data))
- (when (instance? ICredentialsListener spout-obj) (.setCredentials spout-obj initial-credentials))
-
- (.open spout-obj
- storm-conf
- (:user-context task-data)
- (SpoutOutputCollector.
- (reify ISpoutOutputCollector
- (^long getPendingCount[this]
- (.size pending)
- )
- (^List emit [this ^String stream-id ^List tuple ^Object message-id]
- (send-spout-msg stream-id tuple message-id nil)
- )
- (^void emitDirect [this ^int out-task-id ^String stream-id
- ^List tuple ^Object message-id]
- (send-spout-msg stream-id tuple message-id out-task-id)
- )
- (reportError [this error]
- (report-error error)
- )))))
- (reset! open-or-prepare-was-called? true)
- (log-message "Opened spout " component-id ":" (keys task-datas))
- (setup-metrics! executor-data)
-
- (fn []
- ;; This design requires that spouts be non-blocking
- (disruptor/consume-batch receive-queue event-handler)
-
- (let [active? @(:storm-active-atom executor-data)
- curr-count (.get emitted-count)
- backpressure-enabled ((:storm-conf executor-data) TOPOLOGY-BACKPRESSURE-ENABLE)
- throttle-on (and backpressure-enabled
- @(:throttle-on (:worker executor-data)))
- reached-max-spout-pending (and max-spout-pending
- (>= (.size pending) max-spout-pending))
- ]
- (if active?
- ; activated
- (do
- (when-not @last-active
- (reset! last-active true)
- (log-message "Activating spout " component-id ":" (keys task-datas))
- (fast-list-iter [^ISpout spout spouts] (.activate spout)))
-
- (if (and (not (.isFull transfer-queue))
- (not throttle-on)
- (not reached-max-spout-pending))
- (fast-list-iter [^ISpout spout spouts] (.nextTuple spout))))
- ; deactivated
- (do
- (when @last-active
- (reset! last-active false)
- (log-message "Deactivating spout " component-id ":" (keys task-datas))
- (fast-list-iter [^ISpout spout spouts] (.deactivate spout)))
- ;; TODO: log that it's getting throttled
- (Time/sleep 100)
- (builtin-metrics/skipped-inactive! (:spout-throttling-metrics executor-data) (:stats executor-data))))
-
- (if (and (= curr-count (.get emitted-count)) active?)
- (do (.increment empty-emit-streak)
- (.emptyEmit spout-wait-strategy (.get empty-emit-streak))
- ;; update the spout throttling metrics
- (if throttle-on
- (builtin-metrics/skipped-throttle! (:spout-throttling-metrics executor-data) (:stats executor-data))
- (if reached-max-spout-pending
- (builtin-metrics/skipped-max-spout! (:spout-throttling-metrics executor-data) (:stats executor-data)))))
- (.set empty-emit-streak 0)
- ))
- 0))
- :kill-fn (:report-error-and-die executor-data)
- :factory? true
- :thread-name (str component-id "-executor" (:executor-id executor-data)))]))
+ empty-emit-streak (MutableLong. 0)
+ spout-transfer-fn (fn []
+ ;; If topology was started in inactive state, don't call (.open spout) until it's activated first.
+ (while (not @(:storm-active-atom executor-data))
+ (Thread/sleep 100))
+ (log-message "Opening spout " component-id ":" (keys task-datas))
+ (builtin-metrics/register-spout-throttling-metrics (:spout-throttling-metrics executor-data) storm-conf (:user-context (first (vals task-datas))))
+ (doseq [[task-id task-data] task-datas
+ :let [^ISpout spout-obj (:object task-data)
+ tasks-fn (:tasks-fn task-data)
+ send-spout-msg (fn [out-stream-id values message-id out-task-id]
+ (.increment emitted-count)
+ (let [out-tasks (if out-task-id
+ (tasks-fn out-task-id out-stream-id values)
+ (tasks-fn out-stream-id values))
+ rooted? (and message-id has-ackers?)
+ root-id (if rooted? (MessageId/generateId rand))
+ ^List out-ids (fast-list-for [t out-tasks] (if rooted? (MessageId/generateId rand)))]
+ (fast-list-iter [out-task out-tasks id out-ids]
+ (let [tuple-id (if rooted?
+ (MessageId/makeRootId root-id id)
+ (MessageId/makeUnanchored))
+ out-tuple (TupleImpl. worker-context
+ values
+ task-id
+ out-stream-id
+ tuple-id)]
+ (transfer-fn out-task out-tuple)))
+ (if has-eventloggers?
+ (send-to-eventlogger executor-data task-data values component-id message-id rand))
+ (if (and rooted?
+ (not (.isEmpty out-ids)))
+ (do
+ (.put pending root-id [task-id
+ message-id
+ {:stream out-stream-id
+ :values (if debug? values nil)}
+ (if (sampler) (System/currentTimeMillis))])
+ (task/send-unanchored task-data
+ ACKER-INIT-STREAM-ID
+ [root-id (bit-xor-vals out-ids) task-id]))
+ (when message-id
+ (ack-spout-msg executor-data task-data message-id
+ {:stream out-stream-id :values values}
+ (if (sampler) 0) "0:")))
+ (or out-tasks [])))]]
+
+ (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf (:user-context task-data))
+ (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
+ :receive receive-queue}
+ storm-conf (:user-context task-data))
+ (when (instance? ICredentialsListener spout-obj) (.setCredentials spout-obj initial-credentials))
+
+ (.open spout-obj
+ storm-conf
+ (:user-context task-data)
+ (SpoutOutputCollector.
+ (reify ISpoutOutputCollector
+ (^long getPendingCount[this]
+ (.size pending))
+ (^List emit [this ^String stream-id ^List tuple ^Object message-id]
+ (send-spout-msg stream-id tuple message-id nil))
+ (^void emitDirect [this ^int out-task-id ^String stream-id
+ ^List tuple ^Object message-id]
+ (send-spout-msg stream-id tuple message-id out-task-id))
+ (reportError [this error]
+ (report-error error))))))
+
+ (reset! open-or-prepare-was-called? true)
+ (log-message "Opened spout " component-id ":" (keys task-datas))
+ (setup-metrics! executor-data)
+
+ (fn []
+ ;; This design requires that spouts be non-blocking
+ (disruptor/consume-batch receive-queue event-handler)
+
+ (let [active? @(:storm-active-atom executor-data)
+ curr-count (.get emitted-count)
+ backpressure-enabled ((:storm-conf executor-data) TOPOLOGY-BACKPRESSURE-ENABLE)
+ throttle-on (and backpressure-enabled
+ @(:throttle-on (:worker executor-data)))
+ reached-max-spout-pending (and max-spout-pending
+ (>= (.size pending) max-spout-pending))]
+ (if active?
+ ; activated
+ (do
+ (when-not @last-active
+ (reset! last-active true)
+ (log-message "Activating spout " component-id ":" (keys task-datas))
+ (fast-list-iter [^ISpout spout spouts] (.activate spout)))
+
+ (if (and (not (.isFull transfer-queue))
+ (not throttle-on)
+ (not reached-max-spout-pending))
+ (fast-list-iter [^ISpout spout spouts] (.nextTuple spout))))
+ ; deactivated
+ (do
+ (when @last-active
+ (reset! last-active false)
+ (log-message "Deactivating spout " component-id ":" (keys task-datas))
+ (fast-list-iter [^ISpout spout spouts] (.deactivate spout)))
+ ;; TODO: log that it's getting throttled
+ (Time/sleep 100)
+ (builtin-metrics/skipped-inactive! (:spout-throttling-metrics executor-data) (:stats executor-data))))
+
+ (if (and (= curr-count (.get emitted-count)) active?)
+ (do (.increment empty-emit-streak)
+ (.emptyEmit spout-wait-strategy (.get empty-emit-streak))
+ ;; update the spout throttling metrics
+ (if throttle-on
+ (builtin-metrics/skipped-throttle! (:spout-throttling-metrics executor-data) (:stats executor-data))
+ (if reached-max-spout-pending
+ (builtin-metrics/skipped-max-spout! (:spout-throttling-metrics executor-data) (:stats executor-data)))))
+ (.set empty-emit-streak 0)))
+ 0))]
+
+ [(Utils/asyncLoop
+ spout-transfer-fn
+ false ; isDaemon
+ (:report-error-and-die executor-data)
+ Thread/NORM_PRIORITY
+ true ; isFactory
+ true ; startImmediately
+ (str component-id "-executor" (:executor-id executor-data)))]))
(defn- tuple-time-delta! [^TupleImpl tuple]
(let [ms (.getProcessSampleStartTime tuple)]
(if ms
- (time-delta-ms ms))))
+ (Time/deltaMs ms))))
(defn- tuple-execute-time-delta! [^TupleImpl tuple]
(let [ms (.getExecuteSampleStartTime tuple)]
(if ms
- (time-delta-ms ms))))
+ (Time/deltaMs ms))))
(defn put-xor! [^Map pending key id]
(let [curr (or (.get pending key) (long 0))]
@@ -701,7 +710,7 @@
;; TODO: for state sync, need to check if tuple comes from state spout. if so, update state
;; TODO: how to handle incremental updates as well as synchronizations at same time
;; TODO: need to version tuples somehow
-
+
;;(log-debug "Received tuple " tuple " at task " task-id)
;; need to do it this way to avoid reflection
(let [stream-id (.getSourceStreamId tuple)]
@@ -727,122 +736,123 @@
(let [delta (tuple-execute-time-delta! tuple)]
(when (= true (storm-conf TOPOLOGY-DEBUG))
(log-message "Execute done TUPLE " tuple " TASK: " task-id " DELTA: " delta))
-
+
(task/apply-hooks user-context .boltExecute (BoltExecuteInfo. tuple task-id delta))
(when delta
(stats/bolt-execute-tuple! executor-stats
(.getSourceComponent tuple)
(.getSourceStreamId tuple)
delta)))))))
- has-eventloggers? (has-eventloggers? storm-conf)]
-
+ has-eventloggers? (has-eventloggers? storm-conf)
+ bolt-transfer-fn (fn []
+ ;; If topology was started in inactive state, don't call prepare bolt until it's activated first.
+ (while (not @(:storm-active-atom executor-data))
+ (Thread/sleep 100))
+
+ (log-message "Preparing bolt " component-id ":" (keys task-datas))
+ (doseq [[task-id task-data] task-datas
+ :let [^IBolt bolt-obj (:object task-data)
+ tasks-fn (:tasks-fn task-data)
+ user-context (:user-context task-data)
+ bolt-emit (fn [stream anchors values task]
+ (let [out-tasks (if task
+ (tasks-fn task stream values)
+ (tasks-fn stream values))]
+ (fast-list-iter [t out-tasks]
+ (let [anchors-to-ids (HashMap.)]
+ (fast-list-iter [^TupleImpl a anchors]
+ (let [root-ids (-> a .getMessageId .getAnchorsToIds .keySet)]
+ (when (pos? (count root-ids))
+ (let [edge-id (MessageId/generateId rand)]
+ (.updateAckVal a edge-id)
+ (fast-list-iter [root-id root-ids]
+ (put-xor! anchors-to-ids root-id edge-id))))))
+ (let [tuple (TupleImpl. worker-context
+ values
+ task-id
+ stream
+ (MessageId/makeId anchors-to-ids))]
+ (transfer-fn t tuple))))
+ (if has-eventloggers?
+ (send-to-eventlogger executor-data task-data values component-id nil rand))
+ (or out-tasks [])))]]
+ (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
+ (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials))
+ (if (= component-id Constants/SYSTEM_COMPONENT_ID)
+ (do
+ (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
+ :receive (:receive-queue executor-data)
+ :transfer (:transfer-queue (:worker executor-data))}
+ storm-conf user-context)
+ (builtin-metrics/register-iconnection-client-metrics (:cached-node+port->socket (:worker executor-data)) storm-conf user-context)
+ (builtin-metrics/register-iconnection-server-metric (:receiver (:worker executor-data)) storm-conf user-context))
+ (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
+ :receive (:receive-queue executor-data)}
+ storm-conf user-context))
+
+ (.prepare bolt-obj
+ storm-conf
+ user-context
+ (OutputCollector.
+ (reify IOutputCollector
+ (emit [this stream anchors values]
+ (bolt-emit stream anchors values nil))
+ (emitDirect [this task stream anchors values]
+ (bolt-emit stream anchors values task))
+ (^void ack [this ^Tuple tuple]
+ (let [^TupleImpl tuple tuple
+ ack-val (.getAckVal tuple)]
+ (fast-map-iter [[root id] (.. tuple getMessageId getAnchorsToIds)]
+ (task/send-unanchored task-data
+ ACKER-ACK-STREAM-ID
+ [root (bit-xor id ack-val)])))
+ (let [delta (tuple-time-delta! tuple)
+ debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+ (when debug?
+ (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
+ (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
+ (when delta
+ (stats/bolt-acked-tuple! executor-stats
+ (.getSourceComponent tuple)
+ (.getSourceStreamId tuple)
+ delta))))
+ (^void fail [this ^Tuple tuple]
+ (fast-list-iter [root (.. tuple getMessageId getAnchors)]
+ (task/send-unanchored task-data
+ ACKER-FAIL-STREAM-ID
+ [root]))
+ (let [delta (tuple-time-delta! tuple)
+ debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+ (when debug?
+ (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
+ (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
+ (when delta
+ (stats/bolt-failed-tuple! executor-stats
+ (.getSourceComponent tuple)
+ (.getSourceStreamId tuple)
+ delta))))
+ (reportError [this error]
+ (report-error error))))))
+
+ (reset! open-or-prepare-was-called? true)
+ (log-message "Prepared bolt " component-id ":" (keys task-datas))
+ (setup-metrics! executor-data)
+
+ (let [receive-queue (:receive-queue executor-data)
+ event-handler (mk-task-receiver executor-data tuple-action-fn)]
+ (fn []
+ (disruptor/consume-batch-when-available receive-queue event-handler)
+ 0)))]
;; TODO: can get any SubscribedState objects out of the context now
- [(async-loop
- (fn []
- ;; If topology was started in inactive state, don't call prepare bolt until it's activated first.
- (while (not @(:storm-active-atom executor-data))
- (Thread/sleep 100))
-
- (log-message "Preparing bolt " component-id ":" (keys task-datas))
- (doseq [[task-id task-data] task-datas
- :let [^IBolt bolt-obj (:object task-data)
- tasks-fn (:tasks-fn task-data)
- user-context (:user-context task-data)
- bolt-emit (fn [stream anchors values task]
- (let [out-tasks (if task
- (tasks-fn task stream values)
- (tasks-fn stream values))]
- (fast-list-iter [t out-tasks]
- (let [anchors-to-ids (HashMap.)]
- (fast-list-iter [^TupleImpl a anchors]
- (let [root-ids (-> a .getMessageId .getAnchorsToIds .keySet)]
- (when (pos? (count root-ids))
- (let [edge-id (MessageId/generateId rand)]
- (.updateAckVal a edge-id)
- (fast-list-iter [root-id root-ids]
- (put-xor! anchors-to-ids root-id edge-id))
- ))))
- (let [tuple (TupleImpl. worker-context
- values
- task-id
- stream
- (MessageId/makeId anchors-to-ids))]
- (transfer-fn t tuple))))
- (if has-eventloggers?
- (send-to-eventlogger executor-data task-data values component-id nil rand))
- (or out-tasks [])))]]
- (builtin-metrics/register-all (:builtin-metrics task-data) storm-conf user-context)
- (when (instance? ICredentialsListener bolt-obj) (.setCredentials bolt-obj initial-credentials))
- (if (= component-id Constants/SYSTEM_COMPONENT_ID)
- (do
- (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
- :receive (:receive-queue executor-data)
- :transfer (:transfer-queue (:worker executor-data))}
- storm-conf user-context)
- (builtin-metrics/register-iconnection-client-metrics (:cached-node+port->socket (:worker executor-data)) storm-conf user-context)
- (builtin-metrics/register-iconnection-server-metric (:receiver (:worker executor-data)) storm-conf user-context))
- (builtin-metrics/register-queue-metrics {:sendqueue (:batch-transfer-queue executor-data)
- :receive (:receive-queue executor-data)}
- storm-conf user-context)
- )
-
- (.prepare bolt-obj
- storm-conf
- user-context
- (OutputCollector.
- (reify IOutputCollector
- (emit [this stream anchors values]
- (bolt-emit stream anchors values nil))
- (emitDirect [this task stream anchors values]
- (bolt-emit stream anchors values task))
- (^void ack [this ^Tuple tuple]
- (let [^TupleImpl tuple tuple
- ack-val (.getAckVal tuple)]
- (fast-map-iter [[root id] (.. tuple getMessageId getAnchorsToIds)]
- (task/send-unanchored task-data
- ACKER-ACK-STREAM-ID
- [root (bit-xor id ack-val)])))
- (let [delta (tuple-time-delta! tuple)
- debug? (= true (storm-conf TOPOLOGY-DEBUG))]
- (when debug?
- (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
- (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
- (when delta
- (stats/bolt-acked-tuple! executor-stats
- (.getSourceComponent tuple)
- (.getSourceStreamId tuple)
- delta))))
- (^void fail [this ^Tuple tuple]
- (fast-list-iter [root (.. tuple getMessageId getAnchors)]
- (task/send-unanchored task-data
- ACKER-FAIL-STREAM-ID
- [root]))
- (let [delta (tuple-time-delta! tuple)
- debug? (= true (storm-conf TOPOLOGY-DEBUG))]
- (when debug?
- (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
- (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
- (when delta
- (stats/bolt-failed-tuple! executor-stats
- (.getSourceComponent tuple)
- (.getSourceStreamId tuple)
- delta))))
- (reportError [this error]
- (report-error error)
- )))))
- (reset! open-or-prepare-was-called? true)
- (log-message "Prepared bolt " component-id ":" (keys task-datas))
- (setup-metrics! executor-data)
-
- (let [receive-queue (:receive-queue executor-data)
- event-handler (mk-task-receiver executor-data tuple-action-fn)]
- (fn []
- (disruptor/consume-batch-when-available receive-queue event-handler)
- 0)))
- :kill-fn (:report-error-and-die executor-data)
- :factory? true
- :thread-name (str component-id "-executor" (:executor-id executor-data)))]))
+ [(Utils/asyncLoop
+ bolt-transfer-fn
+ false ; isDaemon
+ (:report-error-and-die executor-data)
+ Thread/NORM_PRIORITY
+ true ; isFactory
+ true ; startImmediately
+ (str component-id "-executor" (:executor-id executor-data)))]))
(defmethod close-component :spout [executor-data spout]
(.close spout))
diff --git a/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj b/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj
index 0edfe085a1f..6ca1759911c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/logviewer.clj
@@ -20,7 +20,7 @@
(:use [hiccup core page-helpers form-helpers])
(:use [org.apache.storm config util log timer])
(:use [org.apache.storm.ui helpers])
- (:import [org.apache.storm.utils Utils VersionInfo ConfigUtils])
+ (:import [org.apache.storm.utils Utils Time VersionInfo ConfigUtils])
(:import [org.slf4j LoggerFactory])
(:import [java.util Arrays ArrayList HashSet])
(:import [java.util.zip GZIPInputStream])
@@ -28,10 +28,10 @@
(:import [org.apache.logging.log4j.core Appender LoggerContext])
(:import [org.apache.logging.log4j.core.appender RollingFileAppender])
(:import [java.io BufferedInputStream File FileFilter FileInputStream
- InputStream InputStreamReader])
+ InputStream InputStreamReader]
+ [java.net URLDecoder])
(:import [java.nio.file Files Path Paths DirectoryStream])
(:import [java.nio ByteBuffer])
- (:import [org.apache.storm.utils Utils])
(:import [org.apache.storm.daemon DirectoryCleaner])
(:import [org.yaml.snakeyaml Yaml]
[org.yaml.snakeyaml.constructor SafeConstructor])
@@ -51,6 +51,8 @@
(def ^:dynamic *STORM-CONF* (clojurify-structure (ConfigUtils/readStormConfig)))
(def STORM-VERSION (VersionInfo/getVersion))
+(def worker-log-filename-pattern #"^worker.log(.*)")
+
(defmeter logviewer:num-log-page-http-requests)
(defmeter logviewer:num-daemonlog-page-http-requests)
(defmeter logviewer:num-download-log-file-http-requests)
@@ -117,9 +119,9 @@
(defn get-topo-port-workerlog
"Return the path of the worker log with the format of topoId/port/worker.log.*"
[^File file]
- (clojure.string/join file-path-separator
+ (clojure.string/join Utils/FILE_PATH_SEPARATOR
(take-last 3
- (split (.getCanonicalPath file) (re-pattern file-path-separator)))))
+ (split (.getCanonicalPath file) (re-pattern Utils/FILE_PATH_SEPARATOR)))))
(defn get-metadata-file-for-log-root-name [root-name root-dir]
(let [metaFile (clojure.java.io/file root-dir "metadata"
@@ -141,10 +143,10 @@
nil))))
(defn get-worker-id-from-metadata-file [metaFile]
- (get (clojure-from-yaml-file metaFile) "worker-id"))
+ (get (clojurify-structure (Utils/readYamlFile metaFile)) "worker-id"))
(defn get-topo-owner-from-metadata-file [metaFile]
- (get (clojure-from-yaml-file metaFile) TOPOLOGY-SUBMITTER-USER))
+ (get (clojurify-structure (Utils/readYamlFile metaFile)) TOPOLOGY-SUBMITTER-USER))
(defn identify-worker-log-dirs [log-dirs]
"return the workerid to worker-log-dir map"
@@ -188,7 +190,7 @@
"Return a sorted set of java.io.Files that were written by workers that are
now active"
[conf root-dir]
- (let [alive-ids (get-alive-ids conf (current-time-secs))
+ (let [alive-ids (get-alive-ids conf (Time/currentTimeSecs))
log-dirs (get-all-worker-dirs root-dir)
id->dir (identify-worker-log-dirs log-dirs)]
(apply sorted-set
@@ -227,12 +229,12 @@
[^File dir]
(let [topodir (.getParentFile dir)]
(if (empty? (.listFiles topodir))
- (rmr (.getCanonicalPath topodir)))))
+ (Utils/forceDelete (.getCanonicalPath topodir)))))
(defn cleanup-fn!
"Delete old log dirs for which the workers are no longer alive"
[log-root-dir]
- (let [now-secs (current-time-secs)
+ (let [now-secs (Time/currentTimeSecs)
old-log-dirs (select-dirs-for-cleanup *STORM-CONF*
(* now-secs 1000)
log-root-dir)
@@ -250,7 +252,7 @@
(dofor [dir dead-worker-dirs]
(let [path (.getCanonicalPath dir)]
(log-message "Cleaning up: Removing " path)
- (try (rmr path)
+ (try (Utils/forceDelete path)
(cleanup-empty-topodir! dir)
(catch Exception ex (log-error ex)))))
(per-workerdir-cleanup! (File. log-root-dir) (* per-dir-size (* 1024 1024)) cleaner)
@@ -264,7 +266,7 @@
(schedule-recurring (mk-timer :thread-name "logviewer-cleanup"
:kill-fn (fn [t]
(log-error t "Error when doing logs cleanup")
- (exit-process! 20 "Error when doing log cleanup")))
+ (Utils/exitProcess 20 "Error when doing log cleanup")))
0 ;; Start immediately.
interval-secs
(fn [] (cleanup-fn! log-root-dir))))))
@@ -309,7 +311,7 @@
(defn get-log-user-group-whitelist [fname]
(let [wl-file (ConfigUtils/getLogMetaDataFile fname)
- m (clojure-from-yaml-file wl-file)]
+ m (clojurify-structure (Utils/readYamlFile wl-file))]
(if (not-nil? m)
(do
(let [user-wl (.get m LOGS-USERS)
@@ -514,9 +516,9 @@
(defn url-to-match-centered-in-log-page
[needle fname offset port]
- (let [host (local-hostname)
+ (let [host (Utils/localHostname)
port (logviewer-port)
- fname (clojure.string/join file-path-separator (take-last 3 (split fname (re-pattern file-path-separator))))]
+ fname (clojure.string/join Utils/FILE_PATH_SEPARATOR (take-last 3 (split fname (re-pattern Utils/FILE_PATH_SEPARATOR))))]
(url (str "http://" host ":" port "/log")
{:file fname
:start (max 0
@@ -851,7 +853,7 @@
new-matches (conj matches
(merge these-matches
{ "fileName" file-name
- "port" (first (take-last 2 (split (.getCanonicalPath (first logs)) (re-pattern file-path-separator))))}))
+ "port" (first (take-last 2 (split (.getCanonicalPath (first logs)) (re-pattern Utils/FILE_PATH_SEPARATOR))))}))
new-count (+ match-count (count (these-matches "matches")))]
(if (empty? these-matches)
(recur matches (rest logs) 0 (+ file-offset 1) match-count)
@@ -874,12 +876,12 @@
(defn deep-search-logs-for-topology
[topology-id user ^String root-dir search num-matches port file-offset offset search-archived? callback origin]
(json-response
- (if (or (not search) (not (.exists (File. (str root-dir file-path-separator topology-id)))))
+ (if (or (not search) (not (.exists (File. (str root-dir Utils/FILE_PATH_SEPARATOR topology-id)))))
[]
(let [file-offset (if file-offset (Integer/parseInt file-offset) 0)
offset (if offset (Integer/parseInt offset) 0)
num-matches (or (Integer/parseInt num-matches) 1)
- port-dirs (vec (.listFiles (File. (str root-dir file-path-separator topology-id))))
+ port-dirs (vec (.listFiles (File. (str root-dir Utils/FILE_PATH_SEPARATOR topology-id))))
logs-for-port-fn (partial logs-for-port user)]
(if (or (not port) (= "*" port))
;; Check for all ports
@@ -892,7 +894,7 @@
;; Check just the one port
(if (not (contains? (into #{} (map str (*STORM-CONF* SUPERVISOR-SLOTS-PORTS))) port))
[]
- (let [port-dir (File. (str root-dir file-path-separator topology-id file-path-separator port))]
+ (let [port-dir (File. (str root-dir Utils/FILE_PATH_SEPARATOR topology-id Utils/FILE_PATH_SEPARATOR port))]
(if (or (not (.exists port-dir)) (empty? (logs-for-port user port-dir)))
[]
(let [filtered-logs (logs-for-port user port-dir)]
@@ -945,7 +947,7 @@
(if (= (str port) (.getName port-dir))
(into [] (DirectoryCleaner/getFilesForDir port-dir))))))))
(if (nil? port)
- (let [topo-dir (File. (str log-root file-path-separator topoId))]
+ (let [topo-dir (File. (str log-root Utils/FILE_PATH_SEPARATOR topoId))]
(if (.exists topo-dir)
(reduce concat
(for [port-dir (.listFiles topo-dir)]
@@ -982,7 +984,7 @@
user (.getUserName http-creds-handler servlet-request)
start (if (:start m) (parse-long-from-map m :start))
length (if (:length m) (parse-long-from-map m :length))
- file (url-decode (:file m))]
+ file (URLDecoder/decode (:file m))]
(log-template (log-page file start length (:grep m) user log-root)
file user))
(catch InvalidRequestException ex
@@ -993,21 +995,21 @@
(let [user (.getUserName http-creds-handler servlet-request)
port (second (split host-port #":"))
dir (File. (str log-root
- file-path-separator
+ Utils/FILE_PATH_SEPARATOR
topo-id
- file-path-separator
+ Utils/FILE_PATH_SEPARATOR
port))
file (File. (str log-root
- file-path-separator
+ Utils/FILE_PATH_SEPARATOR
topo-id
- file-path-separator
+ Utils/FILE_PATH_SEPARATOR
port
- file-path-separator
+ Utils/FILE_PATH_SEPARATOR
filename))]
(if (and (.exists dir) (.exists file))
(if (or (blank? (*STORM-CONF* UI-FILTER))
(authorized-log-user? user
- (str topo-id file-path-separator port file-path-separator "worker.log")
+ (str topo-id Utils/FILE_PATH_SEPARATOR port Utils/FILE_PATH_SEPARATOR "worker.log")
*STORM-CONF*))
(-> (resp/response file)
(resp/content-type "application/octet-stream"))
@@ -1019,14 +1021,14 @@
(let [user (.getUserName http-creds-handler servlet-request)
port (second (split host-port #":"))
dir (File. (str log-root
- file-path-separator
+ Utils/FILE_PATH_SEPARATOR
topo-id
- file-path-separator
+ Utils/FILE_PATH_SEPARATOR
port))]
(if (.exists dir)
(if (or (blank? (*STORM-CONF* UI-FILTER))
(authorized-log-user? user
- (str topo-id file-path-separator port file-path-separator "worker.log")
+ (str topo-id Utils/FILE_PATH_SEPARATOR port Utils/FILE_PATH_SEPARATOR "worker.log")
*STORM-CONF*))
(html4
[:head
@@ -1050,7 +1052,7 @@
user (.getUserName http-creds-handler servlet-request)
start (if (:start m) (parse-long-from-map m :start))
length (if (:length m) (parse-long-from-map m :length))
- file (url-decode (:file m))]
+ file (URLDecoder/decode (:file m))]
(log-template (daemonlog-page file start length (:grep m) user daemonlog-root)
file user))
(catch InvalidRequestException ex
@@ -1078,7 +1080,7 @@
;; filter is configured.
(try
(let [user (.getUserName http-creds-handler servlet-request)]
- (search-log-file (url-decode file)
+ (search-log-file (URLDecoder/decode file)
user
(if (= (:is-daemon m) "yes") daemonlog-root log-root)
(:search-string m)
@@ -1192,7 +1194,7 @@
(let [conf (clojurify-structure (ConfigUtils/readStormConfig))
log-root (ConfigUtils/workerArtifactsRoot conf)
daemonlog-root (log-root-dir (conf LOGVIEWER-APPENDER-NAME))]
- (setup-default-uncaught-exception-handler)
+ (Utils/setupDefaultUncaughtExceptionHandler)
(start-log-cleaner! conf log-root)
(log-message "Starting logviewer server for storm version '"
STORM-VERSION
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index f8bf846adea..710cd835224 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -33,12 +33,13 @@
(: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])
+ (:import [org.apache.storm.security.auth ThriftServer ThriftConnectionType ReqContext AuthUtils]
+ [org.apache.storm.logging ThriftAccessLogger])
(: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.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
+ (:import [org.apache.storm.utils TimeCacheMap Time TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
BufferFileInputStream BufferInputStream])
(:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
@@ -59,7 +60,8 @@
(: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])
+ (:import [org.apache.storm.utils VersionInfo]
+ [org.json.simple JSONValue])
(:require [clj-time.core :as time])
(:require [clj-time.coerce :as coerce])
(:require [metrics.meters :refer [defmeter mark!]])
@@ -116,7 +118,7 @@
(conf STORM-SCHEDULER)
(do (log-message "Using custom scheduler: " (conf STORM-SCHEDULER))
- (-> (conf STORM-SCHEDULER) new-instance))
+ (-> (conf STORM-SCHEDULER) Utils/newInstance))
:else
(do (log-message "Using default scheduler")
@@ -159,7 +161,7 @@
(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))]
+ (let [instance (Utils/newInstance (conf NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN))]
(try
(.prepare instance conf)
instance
@@ -189,11 +191,11 @@
: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))
+ :uptime (Utils/makeUptimeComputer)
+ :validator (Utils/newInstance (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")
+ (Utils/exitProcess 20 "Error when processing an event")
))
:scheduler (mk-scheduler conf inimbus)
:leader-elector (Zookeeper/zkLeaderElector conf)
@@ -256,6 +258,10 @@
:topology-action-options {:delay-secs delay :action :kill}})
))
+(defn assoc-non-nil
+ [m k v]
+ (if v (assoc m k v) m))
+
(defn rebalance-transition [nimbus storm-id status]
(fn [time num-workers executor-overrides]
(let [delay (if time
@@ -344,7 +350,7 @@
", status: " status,
" storm-id: " storm-id)]
(if error-on-no-transition?
- (throw-runtime msg)
+ (throw (RuntimeException. msg))
(do (when-not (contains? system-events event)
(log-message msg))
nil))
@@ -408,7 +414,7 @@
[storm-cluster-state]
(let [assignments (.assignments storm-cluster-state nil)]
- (defaulted
+ (or
(apply merge-with set/union
(for [a assignments
[_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
@@ -503,7 +509,7 @@
(> min-replication-count @current-replication-count-conf))
(or (neg? max-replication-wait-time)
(< @total-wait-time max-replication-wait-time)))
- (sleep-secs 1)
+ (Time/sleepSecs 1)
(log-debug "waiting for desired replication to be achieved.
min-replication-count = " min-replication-count " max-replication-wait-time = " max-replication-wait-time
(if (not (ConfigUtils/isLocalMode conf))"current-replication-count for jar key = " @current-replication-count-jar)
@@ -537,6 +543,7 @@
(Utils/fromCompressedJsonConf
(.readBlob blob-store (ConfigUtils/masterStormConfKey storm-id) nimbus-subject))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn read-topology-details [nimbus storm-id]
(let [blob-store (:blob-store nimbus)
storm-base (or
@@ -566,12 +573,12 @@
:else 0)
nimbus-time (if (or (not last-nimbus-time)
(not= last-reported-time reported-time))
- (current-time-secs)
+ (Time/currentTimeSecs)
last-nimbus-time
)]
{:is-timed-out (and
nimbus-time
- (>= (time-delta nimbus-time) timeout))
+ (>= (Time/deltaSecs nimbus-time) timeout))
:nimbus-time nimbus-time
:executor-reported-time reported-time
:heartbeat hb}))
@@ -619,7 +626,7 @@
is-timed-out (-> heartbeats-cache (get executor) :is-timed-out)]
(if (and start-time
(or
- (< (time-delta start-time)
+ (< (Time/deltaSecs start-time)
(conf NIMBUS-TASK-LAUNCH-SECS))
(not is-timed-out)
))
@@ -634,6 +641,7 @@
(defn- to-executor-id [task-ids]
[(first task-ids) (last task-ids)])
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- compute-executors [nimbus storm-id]
(let [conf (:conf nimbus)
blob-store (:blob-store nimbus)
@@ -643,10 +651,12 @@
topology (read-storm-topology-as-nimbus storm-id blob-store)
task->component (storm-task-info topology storm-conf)]
(->> (storm-task-info topology storm-conf)
- reverse-map
+ (Utils/reverseMap)
+ clojurify-structure
(map-val sort)
- (join-maps component->executors)
- (map-val (partial apply partition-fixed))
+ ((fn [ & maps ] (Utils/joinMaps (into-array (into [component->executors] maps)))))
+ (clojurify-structure)
+ (map-val (partial apply (fn part-fixed [a b] (Utils/partitionFixed a b))))
(mapcat second)
(map to-executor-id)
)))
@@ -736,6 +746,7 @@
[sid (SupervisorDetails. sid nil ports)]))
)))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- compute-topology->executor->node+port [scheduler-assignments]
"convert {topology-id -> SchedulerAssignment} to
{topology-id -> {executor [node port]}}"
@@ -773,6 +784,7 @@
(count (.getSlots scheduler-assignment))
0 ))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(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]}}
@@ -857,11 +869,16 @@
_ (reset! (:id->resources nimbus) (.getTopologyResourcesMap cluster))]
(.getAssignments cluster)))
+(defn- map-diff
+ "Returns mappings in m2 that aren't in m1"
+ [m1 m2]
+ (into {} (filter (fn [[k v]] (not= v (m1 k))) m2)))
+
(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)
+ slot-assigned (clojurify-structure (Utils/reverseMap executor->node+port))
+ new-slot-assigned (clojurify-structure (Utils/reverseMap new-executor->node+port))
brand-new-slots (map-diff slot-assigned new-slot-assigned)]
(apply concat (vals brand-new-slots))
))
@@ -919,7 +936,7 @@
topology->executor->node+port (merge (into {} (for [id assigned-topology-ids] {id nil})) topology->executor->node+port)
new-assigned-worker->resources (convert-assignments-to-worker->resources new-scheduler-assignments)
- now-secs (current-time-secs)
+ now-secs (Time/currentTimeSecs)
basic-supervisor-details-map (basic-supervisor-details-map storm-cluster-state)
@@ -975,6 +992,7 @@
(catch Exception e
(log-warn-error e "Ignoring exception from Topology action notifier for storm-Id " storm-id))))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- start-storm [nimbus storm-name storm-id topology-initial-status]
{:pre [(#{:active :inactive} topology-initial-status)]}
(let [storm-cluster-state (:storm-cluster-state nimbus)
@@ -987,7 +1005,7 @@
(.activate-storm! storm-cluster-state
storm-id
(StormBase. storm-name
- (current-time-secs)
+ (Time/currentTimeSecs)
{:type topology-initial-status}
(storm-conf TOPOLOGY-WORKERS)
num-executors
@@ -1021,7 +1039,7 @@
impersonation-authorizer (:impersonation-authorization-handler nimbus)
ctx (or context (ReqContext/context))
check-conf (if storm-conf storm-conf (if storm-name {TOPOLOGY-NAME storm-name}))]
- (log-thrift-access (.requestID ctx) (.remoteAddress ctx) (.principal ctx) operation)
+ (ThriftAccessLogger/logAccess (.requestID ctx) (.remoteAddress ctx) (.principal ctx) operation)
(if (.isImpersonating ctx)
(do
(log-warn "principal: " (.realPrincipal ctx) " is trying to impersonate principal: " (.principal ctx))
@@ -1080,7 +1098,7 @@
(.get_common component)
(->> {TOPOLOGY-TASKS (component-parallelism storm-conf component)}
(merge (component-conf component))
- to-json )))
+ JSONValue/toJSONString)))
ret ))
(defn normalize-conf [conf storm-conf ^StormTopology topology]
@@ -1089,7 +1107,8 @@
(let [component-confs (map
#(-> (ThriftTopologyUtils/getComponentCommon topology %)
.get_json_conf
- from-json)
+ ((fn [c] (if c (JSONValue/parse c))))
+ clojurify-structure)
(ThriftTopologyUtils/getComponentIds topology))
total-conf (merge conf storm-conf)
@@ -1135,17 +1154,17 @@
(log-message "Cleaning up " id)
(.teardown-heartbeats! storm-cluster-state id)
(.teardown-topology-errors! storm-cluster-state id)
- (rmr (ConfigUtils/masterStormDistRoot conf id))
+ (Utils/forceDelete (ConfigUtils/masterStormDistRoot conf id))
(blob-rm-topology-keys id blob-store storm-cluster-state)
(swap! (:heartbeats-cache nimbus) dissoc id)))))
(log-message "not a leader, skipping cleanup")))
(defn- file-older-than? [now seconds file]
- (<= (+ (.lastModified file) (to-millis seconds)) (to-millis now)))
+ (<= (+ (.lastModified file) (Time/secsToMillis seconds)) (Time/secsToMillis now)))
(defn clean-inbox [dir-location seconds]
"Deletes jar files in dir older than seconds."
- (let [now (current-time-secs)
+ (let [now (Time/currentTimeSecs)
pred #(and (.isFile %) (file-older-than? now seconds %))
files (filter pred (file-seq (File. dir-location)))]
(doseq [f files]
@@ -1158,7 +1177,7 @@
"Deletes topologies from history older than minutes."
[mins nimbus]
(locking (:topology-history-lock nimbus)
- (let [cutoff-age (- (current-time-secs) (* mins 60))
+ (let [cutoff-age (- (Time/currentTimeSecs) (* mins 60))
topo-history-state (:topo-history-state nimbus)
curr-history (vec (ls-topo-hist topo-history-state))
new-history (vec (filter (fn [line]
@@ -1255,7 +1274,7 @@
users (ConfigUtils/getTopoLogsUsers topology-conf)
groups (ConfigUtils/getTopoLogsGroups topology-conf)
curr-history (vec (ls-topo-hist topo-history-state))
- new-history (conj curr-history {:topoid storm-id :timestamp (current-time-secs)
+ new-history (conj curr-history {:topoid storm-id :timestamp (Time/currentTimeSecs)
:users users :groups groups})]
(ls-topo-hist! topo-history-state new-history))))
@@ -1309,6 +1328,7 @@
))))))))
(log-message "not a leader skipping , credential renweal.")))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn validate-topology-size [topo-conf nimbus-conf topology]
(let [workers-count (get topo-conf TOPOLOGY-WORKERS)
workers-allowed (get nimbus-conf NIMBUS-SLOTS-PER-TOPOLOGY)
@@ -1352,6 +1372,13 @@
(defmethod blob-sync :local [conf nimbus]
nil)
+(defn- between?
+ "val >= lower and val <= upper"
+ [val lower upper]
+ (and (>= val lower)
+ (<= val upper)))
+
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defserverfn service-handler [conf inimbus]
(.prepare inimbus conf (ConfigUtils/masterInimbusDir conf))
(log-message "Starting Nimbus with conf " conf)
@@ -1401,7 +1428,7 @@
(NimbusSummary.
(.getHost (:nimbus-host-port-info nimbus))
(.getPort (:nimbus-host-port-info nimbus))
- (current-time-secs)
+ (Time/currentTimeSecs)
false ;is-leader
STORM-VERSION))
@@ -1465,7 +1492,8 @@
(validate-topology-name! storm-name)
(check-authorization! nimbus storm-name nil "submitTopology")
(check-storm-active! nimbus storm-name false)
- (let [topo-conf (from-json serializedConf)]
+ (let [topo-conf (if-let [parsed-json (JSONValue/parse serializedConf)]
+ (clojurify-structure parsed-json))]
(try
(ConfigValidation/validateFields topo-conf)
(catch IllegalArgumentException ex
@@ -1475,10 +1503,11 @@
topo-conf
topology))
(swap! (:submitted-count nimbus) inc)
- (let [storm-id (str storm-name "-" @(:submitted-count nimbus) "-" (current-time-secs))
+ (let [storm-id (str storm-name "-" @(:submitted-count nimbus) "-" (Time/currentTimeSecs))
credentials (.get_creds submitOptions)
credentials (when credentials (.get_creds credentials))
- topo-conf (from-json serializedConf)
+ topo-conf (if-let [parsed-json (JSONValue/parse serializedConf)]
+ (clojurify-structure parsed-json))
storm-conf-submitted (normalize-conf
conf
(-> topo-conf
@@ -1514,7 +1543,7 @@
(log-message "Received topology submission for "
storm-name
" with conf "
- (redact-value storm-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
+ (Utils/redactValue storm-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
;; lock protects against multiple topologies being submitted at once and
;; cleanup thread killing topology in b/w assignment and starting the topology
(locking (:submit-lock nimbus)
@@ -1631,6 +1660,7 @@
storm-cluster-state (:storm-cluster-state info)
task->component (:task->component info)
{:keys [executor->node+port node->host]} (:assignment info)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
executor->host+port (map-val (fn [[node port]]
[(node->host node) port])
executor->node+port)
@@ -1685,7 +1715,7 @@
(beginFileUpload [this]
(mark! nimbus:num-beginFileUpload-calls)
(check-authorization! nimbus nil nil "fileUpload")
- (let [fileloc (str (inbox nimbus) "/stormjar-" (uuid) ".jar")]
+ (let [fileloc (str (inbox nimbus) "/stormjar-" (Utils/uuid) ".jar")]
(.put (:uploaders nimbus)
fileloc
(Channels/newChannel (FileOutputStream. fileloc)))
@@ -1725,7 +1755,7 @@
(let [is (BufferInputStream. (.getBlob (:blob-store nimbus) file nil)
^Integer (Utils/getInt (conf STORM-BLOBSTORE-INPUTSTREAM-BUFFER-SIZE-BYTES)
(int 65536)))
- id (uuid)]
+ id (Utils/uuid)]
(.put (:downloaders nimbus) id is)
id))
@@ -1747,7 +1777,7 @@
(^String getNimbusConf [this]
(mark! nimbus:num-getNimbusConf-calls)
(check-authorization! nimbus nil nil "getNimbusConf")
- (to-json (:conf nimbus)))
+ (JSONValue/toJSONString (:conf nimbus)))
(^LogConfig getLogConfig [this ^String id]
(mark! nimbus:num-getLogConfig-calls)
@@ -1763,7 +1793,7 @@
(let [topology-conf (try-read-storm-conf conf id (:blob-store nimbus))
storm-name (topology-conf TOPOLOGY-NAME)]
(check-authorization! nimbus storm-name topology-conf "getTopologyConf")
- (to-json topology-conf)))
+ (JSONValue/toJSONString topology-conf)))
(^StormTopology getTopology [this ^String id]
(mark! nimbus:num-getTopology-calls)
@@ -1793,13 +1823,14 @@
(count ports)
(count (:used-ports info))
id) ]
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(.set_total_resources sup-sum (map-val double (:resources-map info)))
(when-let [[total-mem total-cpu used-mem used-cpu] (.get @(:node-id->resources nimbus) id)]
(.set_used_mem sup-sum used-mem)
(.set_used_cpu sup-sum used-cpu))
(when-let [version (:version info)] (.set_version sup-sum version))
sup-sum))
- nimbus-uptime ((:uptime nimbus))
+ nimbus-uptime (. (:uptime nimbus) upTime)
bases (topology-bases storm-cluster-state)
nimbuses (.nimbuses storm-cluster-state)
@@ -1808,7 +1839,7 @@
leader-host (.getHost leader)
leader-port (.getPort leader)]
(doseq [nimbus-summary nimbuses]
- (.set_uptime_secs nimbus-summary (time-delta (.get_uptime_secs nimbus-summary)))
+ (.set_uptime_secs nimbus-summary (Time/deltaSecs (.get_uptime_secs nimbus-summary)))
(.set_isLeader nimbus-summary (and (= leader-host (.get_host nimbus-summary)) (= leader-port (.get_port nimbus-summary))))))
topology-summaries (dofor [[id base] bases :when base]
@@ -1826,7 +1857,7 @@
vals
set
count)
- (time-delta (:launch-time-secs base))
+ (Time/deltaSecs (:launch-time-secs base))
(extract-status-str base))]
(when-let [owner (:owner base)] (.set_owner topo-summ owner))
(when-let [sched-status (.get @(:id->sched-status nimbus) id)] (.set_sched_status topo-summ sched-status))
@@ -1883,12 +1914,12 @@
(-> executor first task->component)
host
port
- (nil-to-zero (:uptime heartbeat)))
+ (Utils/nullToZero (:uptime heartbeat)))
(.set_stats stats))
))
topo-info (TopologyInfo. storm-id
storm-name
- (time-delta launch-time-secs)
+ (Time/deltaSecs launch-time-secs)
executor-summaries
(extract-status-str base)
errors
@@ -1903,6 +1934,7 @@
(.set_assigned_memoffheap topo-info (get resources 4))
(.set_assigned_cpu topo-info (get resources 5)))
(when-let [component->debug (:component->debug base)]
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(.set_component_debug topo-info (map-val converter/thriftify-debugoptions component->debug)))
(.set_replication_count topo-info (get-blob-replication-count (ConfigUtils/masterStormCodeKey storm-id) nimbus))
topo-info))
@@ -1916,7 +1948,7 @@
(^String beginCreateBlob [this
^String blob-key
^SettableBlobMeta blob-meta]
- (let [session-id (uuid)]
+ (let [session-id (Utils/uuid)]
(.put (:blob-uploaders nimbus)
session-id
(.createBlob (:blob-store nimbus) blob-key blob-meta (get-subject)))
@@ -1927,7 +1959,7 @@
(^String beginUpdateBlob [this ^String blob-key]
(let [^AtomicOutputStream os (.updateBlob (:blob-store nimbus)
blob-key (get-subject))]
- (let [session-id (uuid)]
+ (let [session-id (Utils/uuid)]
(.put (:blob-uploaders nimbus) session-id os)
(log-message "Created upload session for " blob-key
" with id " session-id)
@@ -1951,9 +1983,8 @@
position (.position blob-chunk)]
(.write os chunk-array (+ array-offset position) remaining)
(.put uploaders session os))
- (throw-runtime "Blob for session "
- session
- " does not exist (or timed out)"))))
+ (throw (RuntimeException. (str "Blob for session " session
+ " does not exist (or timed out)"))))))
(^void finishBlobUpload [this ^String session]
(if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
@@ -1963,9 +1994,8 @@
session
". Closing session.")
(.remove (:blob-uploaders nimbus) session))
- (throw-runtime "Blob for session "
- session
- " does not exist (or timed out)")))
+ (throw (RuntimeException. (str "Blob for session " session
+ " does not exist (or timed out)")))))
(^void cancelBlobUpload [this ^String session]
(if-let [^AtomicOutputStream os (.get (:blob-uploaders nimbus) session)]
@@ -1975,9 +2005,8 @@
session
". Closing session.")
(.remove (:blob-uploaders nimbus) session))
- (throw-runtime "Blob for session "
- session
- " does not exist (or timed out)")))
+ (throw (RuntimeException. (str "Blob for session " session
+ " does not exist (or timed out)")))))
(^ReadableBlobMeta getBlobMeta [this ^String blob-key]
(let [^ReadableBlobMeta ret (.getBlobMeta (:blob-store nimbus)
@@ -1992,7 +2021,7 @@
(^BeginDownloadResult beginBlobDownload [this ^String blob-key]
(let [^InputStreamWithMeta is (.getBlob (:blob-store nimbus)
blob-key (get-subject))]
- (let [session-id (uuid)
+ (let [session-id (Utils/uuid)
ret (BeginDownloadResult. (.getVersion is) (str session-id))]
(.set_data_size ret (.getFileLength is))
(.put (:blob-downloaders nimbus) session-id (BufferInputStream. is (Utils/getInt (conf STORM-BLOBSTORE-INPUTSTREAM-BUFFER-SIZE-BYTES) (int 65536))))
@@ -2025,18 +2054,18 @@
(^ListBlobsResult listBlobs [this ^String session]
(let [listers (:blob-listers nimbus)
- ^Iterator keys-it (if (clojure.string/blank? session)
- (.listKeys (:blob-store nimbus))
- (.get listers session))
- _ (or keys-it (throw-runtime "Blob list for session "
- session
- " does not exist (or timed out)"))
-
+ ^Iterator keys-it (or
+ (if (clojure.string/blank? session)
+ (.listKeys (:blob-store nimbus))
+ (.get listers session))
+ (throw (RuntimeException. (str "Blob list for session "
+ session
+ " does not exist (or timed out)"))))
;; Create a new session id if the user gave an empty session string.
;; This is the use case when the user wishes to list blobs
;; starting from the beginning.
session (if (clojure.string/blank? session)
- (let [new-session (uuid)]
+ (let [new-session (Utils/uuid)]
(log-message "Creating new session for downloading list " new-session)
new-session)
session)]
@@ -2095,9 +2124,11 @@
(doto topo-page-info
(.set_name (:storm-name info))
(.set_status (extract-status-str (:base info)))
- (.set_uptime_secs (time-delta (:launch-time-secs info)))
- (.set_topology_conf (to-json (try-read-storm-conf conf
- topo-id (:blob-store nimbus))))
+ (.set_uptime_secs (Time/deltaSecs (:launch-time-secs info)))
+ (.set_topology_conf (JSONValue/toJSONString
+ (try-read-storm-conf conf
+ topo-id
+ (:blob-store nimbus))))
(.set_replication_count (get-blob-replication-count (ConfigUtils/masterStormCodeKey topo-id) nimbus)))
(when-let [debug-options
(get-in info [:base :component->debug topo-id])]
@@ -2115,6 +2146,7 @@
(mark! nimbus:num-getComponentPageInfo-calls)
(let [info (get-common-topo-info topo-id "getComponentPageInfo")
{:keys [executor->node+port node->host]} (:assignment info)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
executor->host+port (map-val (fn [[node port]]
[(node->host node) port])
executor->node+port)
@@ -2138,7 +2170,7 @@
comp-page-info
(converter/thriftify-debugoptions debug-options)))
;; Add the event logger details.
- (let [component->tasks (reverse-map (:task->component info))
+ (let [component->tasks (clojurify-structure (Utils/reverseMap (:task->component info)))
eventlogger-tasks (sort (get component->tasks
EVENTLOGGER-COMPONENT-ID))
;; Find the task the events from this component route to.
@@ -2200,7 +2232,7 @@
(let [service-handler (service-handler conf nimbus)
server (ThriftServer. conf (Nimbus$Processor. service-handler)
ThriftConnectionType/NIMBUS)]
- (add-shutdown-hook-with-force-kill-in-1-sec (fn []
+ (Utils/addShutdownHookWithForceKillIn1Sec (fn []
(.shutdown service-handler)
(.stop server)))
(log-message "Starting nimbus server for storm version '"
@@ -2252,5 +2284,5 @@
))
(defn -main []
- (setup-default-uncaught-exception-handler)
+ (Utils/setupDefaultUncaughtExceptionHandler)
(-launch (standalone-nimbus)))
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 25f89681344..7af2cf0d1ea 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -16,12 +16,13 @@
(ns org.apache.storm.daemon.supervisor
(:import [java.io File IOException FileOutputStream])
(:import [org.apache.storm.scheduler ISupervisor]
- [org.apache.storm.utils LocalState Time Utils ConfigUtils]
+ [org.apache.storm.utils LocalState Time Utils Utils$ExitCodeCallable
+ ConfigUtils]
[org.apache.storm.daemon Shutdownable]
[org.apache.storm Constants]
[org.apache.storm.cluster ClusterStateContext DaemonType]
[java.net JarURLConnection]
- [java.net URI]
+ [java.net URI URLDecoder]
[org.apache.commons.io FileUtils])
(:use [org.apache.storm config util log timer local-state])
(:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
@@ -57,6 +58,7 @@
(shutdown-all-workers [this])
)
+;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
(defn- assignments-snapshot [storm-cluster-state callback assignment-versions]
(let [storm-ids (.assignments storm-cluster-state callback)]
(let [new-assignments
@@ -103,7 +105,7 @@
"Returns map from port to struct containing :storm-id, :executors and :resources"
([assignments-snapshot assignment-id]
(->> (dofor [sid (keys assignments-snapshot)] (read-my-executors assignments-snapshot sid assignment-id))
- (apply merge-with (fn [& ignored] (throw-runtime "Should not have multiple topologies assigned to one port")))))
+ (apply merge-with (fn [& ignored] (throw (RuntimeException. (str "Should not have multiple topologies assigned to one port")))))))
([assignments-snapshot assignment-id existing-assignment retries]
(try (let [assignments (read-assignments assignments-snapshot assignment-id)]
(reset! retries 0)
@@ -113,14 +115,13 @@
(log-warn (.getMessage e) ": retrying " @retries " of 3")
existing-assignment))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- read-storm-code-locations
[assignments-snapshot]
(map-val :master-code-dir assignments-snapshot))
(defn- read-downloaded-storm-ids [conf]
- (let [dir (ConfigUtils/supervisorStormDistRoot conf)]
- (map #(url-decode %) (read-dir-contents dir)))
- )
+ (map #(URLDecoder/decode %) (Utils/readDirContents (ConfigUtils/supervisorStormDistRoot conf))))
(defn read-worker-heartbeat [conf id]
(let [local-state (ConfigUtils/workerState conf id)]
@@ -132,7 +133,7 @@
(defn my-worker-ids [conf]
- (read-dir-contents (ConfigUtils/workerRoot conf)))
+ (Utils/readDirContents (ConfigUtils/workerRoot conf)))
(defn read-worker-heartbeats
"Returns map from worker id to heartbeat"
@@ -199,7 +200,7 @@
(when (and
(not hb)
(<
- (- (current-time-secs) start-time)
+ (- (Time/currentTimeSecs) start-time)
(conf SUPERVISOR-WORKER-START-TIMEOUT-SECS)
))
(log-message id " still hasn't started")
@@ -211,13 +212,13 @@
)))
(defn- wait-for-workers-launch [conf ids]
- (let [start-time (current-time-secs)]
+ (let [start-time (Time/currentTimeSecs)]
(doseq [id ids]
(wait-for-worker-launch conf id start-time))
))
(defn generate-supervisor-id []
- (uuid))
+ (Utils/uuid))
(defnk worker-launcher [conf user args :environment {} :log-prefix nil :exit-code-callback nil :directory nil]
(let [_ (when (clojure.string/blank? user)
@@ -228,13 +229,16 @@
wl (if wl-initial wl-initial (str storm-home "/bin/worker-launcher"))
command (concat [wl user] args)]
(log-message "Running as user:" user " command:" (pr-str command))
- (launch-process command :environment environment :log-prefix log-prefix :exit-code-callback exit-code-callback :directory directory)
- ))
+ (Utils/launchProcess command
+ environment
+ log-prefix
+ exit-code-callback
+ directory)))
(defnk worker-launcher-and-wait [conf user args :environment {} :log-prefix nil]
(let [process (worker-launcher conf user args :environment environment)]
(if log-prefix
- (read-and-log-stream log-prefix (.getInputStream process)))
+ (Utils/readAndLogStream log-prefix (.getInputStream process)))
(try
(.waitFor process)
(catch InterruptedException e
@@ -250,7 +254,7 @@
user
["rmr" path]
:log-prefix (str "rmr " id))
- (if (exists-file? path)
+ (if (Utils/checkFileExists path)
(throw (RuntimeException. (str path " was not deleted"))))))
(defn try-cleanup-worker [conf id]
@@ -260,10 +264,10 @@
(if (conf SUPERVISOR-RUN-WORKER-AS-USER)
(rmr-as-user conf id (ConfigUtils/workerRoot conf id))
(do
- (rmr (ConfigUtils/workerHeartbeatsRoot conf id))
+ (Utils/forceDelete (ConfigUtils/workerHeartbeatsRoot conf id))
;; this avoids a race condition with worker or subprocess writing pid around same time
- (rmr (ConfigUtils/workerPidsRoot conf id))
- (rmr (ConfigUtils/workerRoot conf id))))
+ (Utils/forceDelete (ConfigUtils/workerPidsRoot conf id))
+ (Utils/forceDelete (ConfigUtils/workerRoot conf id))))
(ConfigUtils/removeWorkerUserWSE conf id)
(remove-dead-worker id)
))
@@ -278,7 +282,7 @@
(defn shutdown-worker [supervisor id]
(log-message "Shutting down " (:supervisor-id supervisor) ":" id)
(let [conf (:conf supervisor)
- pids (read-dir-contents (ConfigUtils/workerPidsRoot conf id))
+ pids (Utils/readDirContents (ConfigUtils/workerPidsRoot conf id))
thread-pid (@(:worker-thread-pids-atom supervisor) id)
shutdown-sleep-secs (conf SUPERVISOR-WORKER-SHUTDOWN-SLEEP-SECS)
as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
@@ -288,19 +292,21 @@
(doseq [pid pids]
(if as-user
(worker-launcher-and-wait conf user ["signal" pid "15"] :log-prefix (str "kill -15 " pid))
- (kill-process-with-sig-term pid)))
+ (Utils/killProcessWithSigTerm pid)))
(when-not (empty? pids)
(log-message "Sleep " shutdown-sleep-secs " seconds for execution of cleanup threads on worker.")
- (sleep-secs shutdown-sleep-secs))
+ (Time/sleepSecs shutdown-sleep-secs))
(doseq [pid pids]
(if as-user
(worker-launcher-and-wait conf user ["signal" pid "9"] :log-prefix (str "kill -9 " pid))
- (force-kill-process pid))
- (if as-user
- (rmr-as-user conf id (ConfigUtils/workerPidPath conf id pid))
- (try
- (rmpath (ConfigUtils/workerPidPath conf id pid))
- (catch Exception e)))) ;; on windows, the supervisor may still holds the lock on the worker directory
+ (Utils/forceKillProcess pid))
+ (let [path (ConfigUtils/workerPidPath conf id pid)]
+ (if as-user
+ (rmr-as-user conf id path)
+ (try
+ (log-debug "Removing path " path)
+ (.delete (File. path))
+ (catch Exception e))))) ;; on windows, the supervisor may still holds the lock on the worker directory
(try-cleanup-worker conf id))
(log-message "Shut down " (:supervisor-id supervisor) ":" id))
@@ -313,7 +319,7 @@
:shared-context shared-context
:isupervisor isupervisor
:active (atom true)
- :uptime (uptime-computer)
+ :uptime (Utils/makeUptimeComputer)
:version STORM-VERSION
:worker-thread-pids-atom (atom {})
:storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
@@ -324,20 +330,20 @@
:local-state (ConfigUtils/supervisorState conf)
:supervisor-id (.getSupervisorId isupervisor)
:assignment-id (.getAssignmentId isupervisor)
- :my-hostname (hostname conf)
+ :my-hostname (Utils/hostname conf)
:curr-assignment (atom nil) ;; used for reporting used ports when heartbeating
:heartbeat-timer (mk-timer :kill-fn (fn [t]
(log-error t "Error when processing event")
- (exit-process! 20 "Error when processing an event")
+ (Utils/exitProcess 20 "Error when processing an event")
))
:event-timer (mk-timer :kill-fn (fn [t]
(log-error t "Error when processing event")
- (exit-process! 20 "Error when processing an event")
+ (Utils/exitProcess 20 "Error when processing an event")
))
:blob-update-timer (mk-timer :kill-fn (defn blob-update-timer
[t]
(log-error t "Error when processing event")
- (exit-process! 20 "Error when processing a event"))
+ (Utils/exitProcess 20 "Error when processing a event"))
:timer-name "blob-update-timer")
:localizer (Utils/createLocalizer conf (ConfigUtils/supervisorLocalDir conf))
:assignment-versions (atom {})
@@ -352,9 +358,9 @@
stormjarpath (ConfigUtils/supervisorStormJarPath stormroot)
stormcodepath (ConfigUtils/supervisorStormCodePath stormroot)
stormconfpath (ConfigUtils/supervisorStormConfPath stormroot)]
- (and (every? exists-file? [stormroot stormconfpath stormcodepath])
+ (and (every? #(Utils/checkFileExists %) [stormroot stormconfpath stormcodepath])
(or (ConfigUtils/isLocalMode conf)
- (exists-file? stormjarpath)))))
+ (Utils/checkFileExists stormjarpath)))))
(defn get-worker-assignment-helper-msg
[assignment supervisor port id]
@@ -372,11 +378,12 @@
mem-onheap (.get_mem_on_heap resources)]
;; This condition checks for required files exist before launching the worker
(if (required-topo-files-exist? conf storm-id)
- (do
+ (let [pids-path (ConfigUtils/workerPidsRoot conf id)
+ hb-path (ConfigUtils/workerHeartbeatsRoot conf id)]
(log-message "Launching worker with assignment "
(get-worker-assignment-helper-msg assignment supervisor port id))
- (local-mkdirs (ConfigUtils/workerPidsRoot conf id))
- (local-mkdirs (ConfigUtils/workerHeartbeatsRoot conf id))
+ (FileUtils/forceMkdir (File. pids-path))
+ (FileUtils/forceMkdir (File. hb-path))
(launch-worker supervisor
(:storm-id assignment)
port
@@ -388,12 +395,18 @@
(get-worker-assignment-helper-msg assignment supervisor port id))
nil)))))))
+
+(defn- select-keys-pred
+ [pred amap]
+ (into {} (filter (fn [[k v]] (pred k)) amap)))
+
+;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
(defn sync-processes [supervisor]
(let [conf (:conf supervisor)
^LocalState local-state (:local-state supervisor)
storm-cluster-state (:storm-cluster-state supervisor)
- assigned-executors (defaulted (ls-local-assignments local-state) {})
- now (current-time-secs)
+ assigned-executors (or (ls-local-assignments local-state) {})
+ now (Time/currentTimeSecs)
allocated (read-allocated-workers supervisor assigned-executors now)
keepers (filter-val
(fn [[state _]] (= state :valid))
@@ -403,7 +416,7 @@
new-worker-ids (into
{}
(for [port (keys reassign-executors)]
- [port (uuid)]))]
+ [port (Utils/uuid)]))]
;; 1. to kill are those in allocated that are dead or disallowed
;; 2. kill the ones that should be dead
;; - read pids, kill -9 and individually remove file
@@ -439,11 +452,12 @@
(map :storm-id)
set))
+;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
(defn shutdown-disallowed-workers [supervisor]
(let [conf (:conf supervisor)
^LocalState local-state (:local-state supervisor)
- assigned-executors (defaulted (ls-local-assignments local-state) {})
- now (current-time-secs)
+ assigned-executors (or (ls-local-assignments local-state) {})
+ now (Time/currentTimeSecs)
allocated (read-allocated-workers supervisor assigned-executors now)
disallowed (keys (filter-val
(fn [[state _]] (= state :disallowed))
@@ -508,7 +522,7 @@
(remove-blob-references localizer storm-id conf))
(if (conf SUPERVISOR-RUN-WORKER-AS-USER)
(rmr-as-user conf storm-id path)
- (rmr (ConfigUtils/supervisorStormDistRoot conf storm-id)))
+ (Utils/forceDelete (ConfigUtils/supervisorStormDistRoot conf storm-id)))
(catch Exception e
(log-message e (str "Exception removing: " storm-id))))))
@@ -544,6 +558,7 @@
(:assignment-id supervisor)
existing-assignment
(:sync-retry supervisor))
+ ;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
new-assignment (->> all-assignment
(filter-key #(.confirmAssigned isupervisor %)))
assigned-storm-ids (assigned-storm-ids-from-port-assignments new-assignment)
@@ -593,7 +608,7 @@
;; important that this happens after setting the local assignment so that
;; synchronize-supervisor doesn't try to launch workers for which the
;; resources don't exist
- (if on-windows? (shutdown-disallowed-workers supervisor))
+ (if (Utils/isOnWindows) (shutdown-disallowed-workers supervisor))
(doseq [storm-id all-downloaded-storm-ids]
(when-not (storm-code-map storm-id)
(log-message "Removing code for storm id "
@@ -650,7 +665,7 @@
(let [java-home (.get (System/getenv) "JAVA_HOME")]
(if (nil? java-home)
cmd
- (str java-home file-path-separator "bin" file-path-separator cmd))))
+ (str java-home Utils/FILE_PATH_SEPARATOR "bin" Utils/FILE_PATH_SEPARATOR cmd))))
(defn java-cmd []
(jvm-cmd "java"))
@@ -681,24 +696,24 @@
"Launch profiler action for a worker"
[conf user target-dir command :environment {} :exit-code-on-profile-action nil :log-prefix nil]
(if-let [run-worker-as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)]
- (let [container-file (container-file-path target-dir)
- script-file (script-file-path target-dir)]
- (log-message "Running as user:" user " command:" (shell-cmd command))
- (if (exists-file? container-file) (rmr-as-user conf container-file container-file))
- (if (exists-file? script-file) (rmr-as-user conf script-file script-file))
+ (let [container-file (Utils/containerFilePath target-dir)
+ script-file (Utils/scriptFilePath target-dir)]
+ (log-message "Running as user:" user " command:" (Utils/shellCmd command))
+ (if (Utils/checkFileExists container-file) (rmr-as-user conf container-file container-file))
+ (if (Utils/checkFileExists script-file) (rmr-as-user conf script-file script-file))
(worker-launcher
conf
user
- ["profiler" target-dir (write-script target-dir command :environment environment)]
+ ["profiler" target-dir (Utils/writeScript target-dir command environment)]
:log-prefix log-prefix
:exit-code-callback exit-code-on-profile-action
:directory (File. target-dir)))
- (launch-process
+ (Utils/launchProcess
command
- :environment environment
- :log-prefix log-prefix
- :exit-code-callback exit-code-on-profile-action
- :directory (File. target-dir))))
+ environment
+ log-prefix
+ exit-code-on-profile-action
+ (File. target-dir))))
(defn mk-run-profiler-actions-for-all-topologies
"Returns a function that downloads all profile-actions listed for all topologies assigned
@@ -779,14 +794,14 @@
heartbeat-fn (fn [] (.supervisor-heartbeat!
(:storm-cluster-state supervisor)
(:supervisor-id supervisor)
- (->SupervisorInfo (current-time-secs)
+ (->SupervisorInfo (Time/currentTimeSecs)
(:my-hostname supervisor)
(:assignment-id supervisor)
(keys @(:curr-assignment supervisor))
;; used ports
(.getMetadata isupervisor)
(conf SUPERVISOR-SCHEDULER-META)
- ((:uptime supervisor))
+ (. (:uptime supervisor) upTime)
(:version supervisor)
(mk-supervisor-capacities conf))))]
(heartbeat-fn)
@@ -899,7 +914,7 @@
key-name (.getName rsrc-file-path)
blob-symlink-target-name (.getName (File. (.getCurrentSymlinkPath local-rsrc)))
symlink-name (get-blob-localname (get blobstore-map key-name) key-name)]
- (create-symlink! tmproot (.getParent rsrc-file-path) symlink-name
+ (Utils/createSymlink tmproot (.getParent rsrc-file-path) symlink-name
blob-symlink-target-name))))
(catch AuthorizationException authExp
(log-error authExp))
@@ -926,14 +941,15 @@
(defmethod download-storm-code
:distributed [conf storm-id master-code-dir localizer]
;; Downloading to permanent location is atomic
- (let [tmproot (str (ConfigUtils/supervisorTmpDir conf) file-path-separator (uuid))
+
+ (let [tmproot (str (ConfigUtils/supervisorTmpDir conf) Utils/FILE_PATH_SEPARATOR (Utils/uuid))
stormroot (ConfigUtils/supervisorStormDistRoot conf storm-id)
blobstore (Utils/getClientBlobStoreForSupervisor conf)]
(FileUtils/forceMkdir (File. tmproot))
- (if-not on-windows?
+ (if-not (Utils/isOnWindows)
(Utils/restrictPermissions tmproot)
(if (conf SUPERVISOR-RUN-WORKER-AS-USER)
- (throw-runtime (str "ERROR: Windows doesn't implement setting the correct permissions"))))
+ (throw (RuntimeException. (str "ERROR: Windows doesn't implement setting the correct permissions")))))
(Utils/downloadResourcesAsSupervisor (ConfigUtils/masterStormJarKey storm-id)
(ConfigUtils/supervisorStormJarPath tmproot) blobstore)
(Utils/downloadResourcesAsSupervisor (ConfigUtils/masterStormCodeKey storm-id)
@@ -941,7 +957,7 @@
(Utils/downloadResourcesAsSupervisor (ConfigUtils/masterStormConfKey storm-id)
(ConfigUtils/supervisorStormConfPath tmproot) blobstore)
(.shutdown blobstore)
- (extract-dir-from-jar (ConfigUtils/supervisorStormJarPath tmproot) ConfigUtils/RESOURCES_SUBDIR tmproot)
+ (Utils/extractDirFromJar (ConfigUtils/supervisorStormJarPath tmproot) ConfigUtils/RESOURCES_SUBDIR tmproot)
(download-blobs-for-topology! conf (ConfigUtils/supervisorStormConfPath tmproot) localizer
tmproot)
(if (download-blobs-for-topology-succeed? (ConfigUtils/supervisorStormConfPath tmproot) tmproot)
@@ -953,7 +969,7 @@
(setup-storm-code-dir conf (clojurify-structure (ConfigUtils/readSupervisorStormConf conf storm-id)) stormroot))
(do
(log-message "Failed to download blob resources for storm-id " storm-id)
- (rmr tmproot)))))
+ (Utils/forceDelete tmproot)))))
(defn write-log-metadata-to-yaml-file! [storm-id port data conf]
(let [file (ConfigUtils/getLogMetaDataFile conf storm-id port)]
@@ -1023,9 +1039,9 @@
resource-file-names (cons ConfigUtils/RESOURCES_SUBDIR blob-file-names)]
(log-message "Creating symlinks for worker-id: " worker-id " storm-id: "
storm-id " for files(" (count resource-file-names) "): " (pr-str resource-file-names))
- (create-symlink! workerroot stormroot ConfigUtils/RESOURCES_SUBDIR)
+ (Utils/createSymlink workerroot stormroot ConfigUtils/RESOURCES_SUBDIR)
(doseq [file-name blob-file-names]
- (create-symlink! workerroot stormroot file-name file-name))))
+ (Utils/createSymlink workerroot stormroot file-name file-name))))
(defn create-artifacts-link
"Create a symlink from workder directory to its port artifacts directory"
@@ -1035,7 +1051,7 @@
(log-message "Creating symlinks for worker-id: " worker-id " storm-id: "
storm-id " to its port artifacts directory")
(if (.exists (File. worker-dir))
- (create-symlink! worker-dir topo-dir "artifacts" port))))
+ (Utils/createSymlink worker-dir topo-dir "artifacts" port))))
(defmethod launch-worker
:distributed [supervisor storm-id port worker-id mem-onheap]
@@ -1047,10 +1063,10 @@
storm-log-dir (ConfigUtils/getLogDir)
storm-log-conf-dir (conf STORM-LOG4J2-CONF-DIR)
storm-log4j2-conf-dir (if storm-log-conf-dir
- (if (is-absolute-path? storm-log-conf-dir)
+ (if (.isAbsolute (File. storm-log-conf-dir))
storm-log-conf-dir
- (str storm-home file-path-separator storm-log-conf-dir))
- (str storm-home file-path-separator "log4j2"))
+ (str storm-home Utils/FILE_PATH_SEPARATOR storm-log-conf-dir))
+ (str storm-home Utils/FILE_PATH_SEPARATOR "log4j2"))
stormroot (ConfigUtils/supervisorStormDistRoot conf storm-id)
jlp (jlp stormroot conf)
stormjar (ConfigUtils/supervisorStormJarPath stormroot)
@@ -1058,9 +1074,9 @@
topo-classpath (if-let [cp (storm-conf TOPOLOGY-CLASSPATH)]
[cp]
[])
- classpath (-> (worker-classpath)
- (add-to-classpath [stormjar])
- (add-to-classpath topo-classpath))
+ classpath (-> (Utils/workerClasspath)
+ (Utils/addToClasspath [stormjar])
+ (Utils/addToClasspath topo-classpath))
top-gc-opts (storm-conf TOPOLOGY-WORKER-GC-CHILDOPTS)
mem-onheap (if (and mem-onheap (> mem-onheap 0)) ;; not nil and not zero
(int (Math/ceil mem-onheap)) ;; round up
@@ -1087,7 +1103,7 @@
storm-log4j2-conf-dir
(str "file:///" storm-log4j2-conf-dir))
storm-log4j2-conf-dir)
- file-path-separator "worker.xml")
+ Utils/FILE_PATH_SEPARATOR "worker.xml")
command (concat
[(java-cmd) "-cp" classpath
topo-worker-logwriter-childopts
@@ -1125,34 +1141,40 @@
(:assignment-id supervisor)
port
worker-id])
- command (->> command (map str) (filter (complement empty?)))]
- (log-message "Launching worker with command: " (shell-cmd command))
+ command (->> command
+ (map str)
+ (filter (complement empty?)))]
+ (log-message "Launching worker with command: " (Utils/shellCmd command))
(write-log-metadata! storm-conf user worker-id storm-id port conf)
(ConfigUtils/setWorkerUserWSE conf worker-id user)
(create-artifacts-link conf storm-id port worker-id)
(let [log-prefix (str "Worker Process " worker-id)
- callback (fn [exit-code]
- (log-message log-prefix " exited with code: " exit-code)
- (add-dead-worker worker-id))
+ callback (reify Utils$ExitCodeCallable
+ (call [this exit-code]
+ (log-message log-prefix " exited with code: " exit-code)
+ (add-dead-worker worker-id)))
worker-dir (ConfigUtils/workerRoot conf worker-id)]
(remove-dead-worker worker-id)
(create-blobstore-links conf storm-id worker-id)
(if run-worker-as-user
- (worker-launcher conf user ["worker" worker-dir (write-script worker-dir command :environment topology-worker-environment)] :log-prefix log-prefix :exit-code-callback callback :directory (File. worker-dir))
- (launch-process command :environment topology-worker-environment :log-prefix log-prefix :exit-code-callback callback :directory (File. worker-dir)))
- )))
+ (worker-launcher conf user ["worker" worker-dir (Utils/writeScript worker-dir command topology-worker-environment)] :log-prefix log-prefix :exit-code-callback callback :directory (File. worker-dir))
+ (Utils/launchProcess command
+ topology-worker-environment
+ log-prefix
+ callback
+ (File. worker-dir))))))
;; local implementation
(defn resources-jar []
- (->> (.split (current-classpath) File/pathSeparator)
+ (->> (.split (Utils/currentClasspath) File/pathSeparator)
(filter #(.endsWith % ".jar"))
- (filter #(zip-contains-dir? % ConfigUtils/RESOURCES_SUBDIR))
+ (filter #(Utils/zipDoesContainDir % ConfigUtils/RESOURCES_SUBDIR))
first ))
(defmethod download-storm-code
:local [conf storm-id master-code-dir localizer]
- (let [tmproot (str (ConfigUtils/supervisorTmpDir conf) file-path-separator (uuid))
+ (let [tmproot (str (ConfigUtils/supervisorTmpDir conf) Utils/FILE_PATH_SEPARATOR (Utils/uuid))
stormroot (ConfigUtils/supervisorStormDistRoot conf storm-id)
blob-store (Utils/getNimbusBlobStore conf master-code-dir nil)]
(try
@@ -1166,12 +1188,12 @@
(let [classloader (.getContextClassLoader (Thread/currentThread))
resources-jar (resources-jar)
url (.getResource classloader ConfigUtils/RESOURCES_SUBDIR)
- target-dir (str stormroot file-path-separator ConfigUtils/RESOURCES_SUBDIR)]
+ target-dir (str stormroot Utils/FILE_PATH_SEPARATOR ConfigUtils/RESOURCES_SUBDIR)]
(cond
resources-jar
(do
(log-message "Extracting resources from jar at " resources-jar " to " target-dir)
- (extract-dir-from-jar resources-jar ConfigUtils/RESOURCES_SUBDIR stormroot))
+ (Utils/extractDirFromJar resources-jar ConfigUtils/RESOURCES_SUBDIR stormroot))
url
(do
(log-message "Copying resources at " (str url) " to " target-dir)
@@ -1180,7 +1202,7 @@
(defmethod launch-worker
:local [supervisor storm-id port worker-id mem-onheap]
(let [conf (:conf supervisor)
- pid (uuid)
+ pid (Utils/uuid)
worker (worker/mk-worker conf
(:shared-context supervisor)
storm-id
@@ -1198,7 +1220,7 @@
(let [conf (clojurify-structure (ConfigUtils/readStormConfig))]
(validate-distributed-mode! conf)
(let [supervisor (mk-supervisor conf nil supervisor)]
- (add-shutdown-hook-with-force-kill-in-1-sec #(.shutdown supervisor)))
+ (Utils/addShutdownHookWithForceKillIn1Sec #(.shutdown supervisor)))
(defgauge supervisor:num-slots-used-gauge #(count (my-worker-ids conf)))
(start-metrics-reporters conf)))
@@ -1229,5 +1251,5 @@
))))
(defn -main []
- (setup-default-uncaught-exception-handler)
+ (Utils/setupDefaultUncaughtExceptionHandler)
(-launch (standalone-supervisor)))
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 643bc385cbe..a097e364295 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -76,7 +76,7 @@
(contains? spouts component-id) (.get_spout_object ^SpoutSpec (get spouts component-id))
(contains? bolts component-id) (.get_bolt_object ^Bolt (get bolts component-id))
(contains? state-spouts component-id) (.get_state_spout_object ^StateSpoutSpec (get state-spouts component-id))
- true (throw-runtime "Could not find " component-id " in " topology)))
+ true (throw (RuntimeException. (str "Could not find " component-id " in " topology)))))
obj (if (instance? ShellComponent obj)
(if (contains? spouts component-id)
(ShellSpout. obj)
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 48934f6538e..b2bdcdb7ba0 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -23,9 +23,12 @@
(:require [clojure.set :as set])
(:require [org.apache.storm.messaging.loader :as msg-loader])
(:import [java.util.concurrent Executors]
- [org.apache.storm.hooks IWorkerHook BaseWorkerHook])
- (:import [java.util ArrayList HashMap])
- (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue])
+ [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
+ [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
+ (:import [java.util ArrayList HashMap]
+ [java.util.concurrent.locks ReentrantReadWriteLock])
+ (:import [org.apache.commons.io FileUtils])
+ (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue Time])
(:import [org.apache.storm.grouping LoadMapping])
(:import [org.apache.storm.messaging TransportFactory])
(:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
@@ -68,8 +71,8 @@
(apply merge)))
zk-hb {:storm-id (:storm-id worker)
:executor-stats stats
- :uptime ((:uptime worker))
- :time-secs (current-time-secs)
+ :uptime (. (:uptime worker) upTime)
+ :time-secs (Time/currentTimeSecs)
}]
;; do the zookeeper heartbeat
(try
@@ -81,7 +84,7 @@
(let [conf (:conf worker)
state (ConfigUtils/workerState conf (:worker-id worker))]
;; do the local-file-system heartbeat.
- (ls-worker-heartbeat! state (current-time-secs) (:storm-id worker) (:executors worker) (:port worker))
+ (ls-worker-heartbeat! state (Time/currentTimeSecs) (:storm-id worker) (:executors worker) (:port worker))
(.cleanup state 60) ; this is just in case supervisor is down so that disk doesn't fill up.
; it shouldn't take supervisor 120 seconds between listing dir and reading it
@@ -101,7 +104,8 @@
(:task-ids worker))]
(-> worker
:task->component
- reverse-map
+ (Utils/reverseMap)
+ clojurify-structure
(select-keys components)
vals
flatten
@@ -237,7 +241,7 @@
(defn mk-halting-timer [timer-name]
(mk-timer :kill-fn (fn [t]
(log-error t "Error when processing event")
- (exit-process! 20 "Error when processing an event")
+ (Utils/exitProcess 20 "Error when processing an event")
)
:timer-name timer-name))
@@ -290,19 +294,21 @@
:user-timer (mk-halting-timer "user-timer")
:task->component (HashMap. (storm-task-info topology storm-conf)) ; for optimized access when used in tasks later on
:component->stream->fields (component->stream->fields (:system-topology <>))
- :component->sorted-tasks (->> (:task->component <>) reverse-map (map-val sort))
- :endpoint-socket-lock (mk-rw-lock)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
+ :component->sorted-tasks (->> (:task->component <>) (Utils/reverseMap) (clojurify-structure) (map-val sort))
+ :endpoint-socket-lock (ReentrantReadWriteLock.)
:cached-node+port->socket (atom {})
:cached-task->node+port (atom {})
:transfer-queue transfer-queue
:executor-receive-queue-map executor-receive-queue-map
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:short-executor-receive-queue-map (map-key first executor-receive-queue-map)
:task->short-executor (->> executors
(mapcat (fn [e] (for [t (executor-id->tasks e)] [t (first e)])))
(into {})
(HashMap.))
:suicide-fn (mk-suicide-fn conf)
- :uptime (uptime-computer)
+ :uptime (Utils/makeUptimeComputer)
:default-shared-resources (mk-default-resources <>)
:user-shared-resources (mk-user-resources <>)
:transfer-local-fn (mk-transfer-local-fn <>)
@@ -325,6 +331,7 @@
(def LOAD-REFRESH-INTERVAL-MS 5000)
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn mk-refresh-load [worker]
(let [local-tasks (set (:task-ids worker))
remote-tasks (set/difference (worker-outbound-tasks worker) local-tasks)
@@ -345,6 +352,23 @@
(.sendLoadMetrics (:receiver worker) local-pop)
(reset! next-update (+ LOAD-REFRESH-INTERVAL-MS now))))))))
+(defmacro read-locked
+ [rw-lock & body]
+ (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})]
+ `(let [rlock# (.readLock ~lock)]
+ (try (.lock rlock#)
+ ~@body
+ (finally (.unlock rlock#))))))
+
+(defmacro write-locked
+ [rw-lock & body]
+ (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})]
+ `(let [wlock# (.writeLock ~lock)]
+ (try (.lock wlock#)
+ ~@body
+ (finally (.unlock wlock#))))))
+
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn mk-refresh-connections [worker]
(let [outbound-tasks (worker-outbound-tasks worker)
conf (:conf worker)
@@ -364,8 +388,10 @@
:executor->node+port
to-task->node+port
(select-keys outbound-tasks)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(#(map-val endpoint->string %)))
;; we dont need a connection for the local tasks anymore
+ ;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
needed-assignment (->> my-assignment
(filter-key (complement (-> worker :task-ids set))))
needed-connections (-> needed-assignment vals set)
@@ -578,12 +604,12 @@
(log-message "Launching worker for " storm-id " on " assignment-id ":" port " with id " worker-id
" and conf " conf)
(if-not (ConfigUtils/isLocalMode conf)
- (redirect-stdio-to-slf4j!))
+ (SysOutOverSLF4J/sendSystemOutAndErrToSLF4J))
;; because in local mode, its not a separate
;; process. supervisor will register it in this case
(when (= :distributed (ConfigUtils/clusterMode conf))
- (let [pid (process-pid)]
- (touch (ConfigUtils/workerPidPath conf worker-id pid))
+ (let [pid (Utils/processPid)]
+ (FileUtils/touch (ConfigUtils/workerPidPath conf worker-id pid))
(spit (ConfigUtils/workerArtifactsPidPath conf storm-id port) pid)))
(declare establish-log-setting-callback)
@@ -744,22 +770,22 @@
(schedule-recurring (:reset-log-levels-timer worker) 0 (conf WORKER-LOG-LEVEL-RESET-POLL-SECS) (fn [] (reset-log-levels latest-log-config)))
(schedule-recurring (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
- (log-message "Worker has topology config " (redact-value (:storm-conf worker) STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
+ (log-message "Worker has topology config " (Utils/redactValue (:storm-conf worker) STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
(log-message "Worker " worker-id " for storm " storm-id " on " assignment-id ":" port " has finished loading")
ret
))))))
(defmethod mk-suicide-fn
:local [conf]
- (fn [] (exit-process! 1 "Worker died")))
+ (fn [] (Utils/exitProcess 1 "Worker died")))
(defmethod mk-suicide-fn
:distributed [conf]
- (fn [] (exit-process! 1 "Worker died")))
+ (fn [] (Utils/exitProcess 1 "Worker died")))
(defn -main [storm-id assignment-id port-str worker-id]
(let [conf (clojurify-structure (ConfigUtils/readStormConfig))]
- (setup-default-uncaught-exception-handler)
+ (Utils/setupDefaultUncaughtExceptionHandler)
(validate-distributed-mode! conf)
(let [worker (mk-worker conf nil storm-id assignment-id (Integer/parseInt port-str) worker-id)]
- (add-shutdown-hook-with-force-kill-in-1-sec #(.shutdown worker)))))
+ (Utils/addShutdownHookWithForceKillIn1Sec #(.shutdown worker)))))
diff --git a/storm-core/src/clj/org/apache/storm/disruptor.clj b/storm-core/src/clj/org/apache/storm/disruptor.clj
index 1546b3ffd37..e2211c0a401 100644
--- a/storm-core/src/clj/org/apache/storm/disruptor.clj
+++ b/storm-core/src/clj/org/apache/storm/disruptor.clj
@@ -15,7 +15,7 @@
;; limitations under the License.
(ns org.apache.storm.disruptor
- (:import [org.apache.storm.utils DisruptorQueue WorkerBackpressureCallback DisruptorBackpressureCallback])
+ (:import [org.apache.storm.utils DisruptorQueue WorkerBackpressureCallback DisruptorBackpressureCallback Utils])
(:import [com.lmax.disruptor.dsl ProducerType])
(:require [clojure [string :as str]])
(:require [clojure [set :as set]])
@@ -78,11 +78,11 @@
(defnk consume-loop*
[^DisruptorQueue queue handler
- :kill-fn (fn [error] (exit-process! 1 "Async loop died!"))]
- (async-loop
+ :uncaught-exception-handler nil]
+ (Utils/asyncLoop
(fn [] (consume-batch-when-available queue handler) 0)
- :kill-fn kill-fn
- :thread-name (.getName queue)))
+ (.getName queue)
+ uncaught-exception-handler))
(defmacro consume-loop [queue & handler-args]
`(let [handler# (handler ~@handler-args)]
diff --git a/storm-core/src/clj/org/apache/storm/event.clj b/storm-core/src/clj/org/apache/storm/event.clj
index edc7616f228..60c22c6a6f6 100644
--- a/storm-core/src/clj/org/apache/storm/event.clj
+++ b/storm-core/src/clj/org/apache/storm/event.clj
@@ -45,7 +45,7 @@
(log-message "Event manager interrupted"))
(catch Throwable t
(log-error t "Error when processing event")
- (exit-process! 20 "Error when processing an event")))))]
+ (Utils/exitProcess 20 "Error when processing an event")))))]
(.setDaemon runner daemon?)
(.start runner)
(reify
diff --git a/storm-core/src/clj/org/apache/storm/local_state.clj b/storm-core/src/clj/org/apache/storm/local_state.clj
index a95a85be0b0..df67c5eb368 100644
--- a/storm-core/src/clj/org/apache/storm/local_state.clj
+++ b/storm-core/src/clj/org/apache/storm/local_state.clj
@@ -21,7 +21,8 @@
LSSupervisorAssignments LocalAssignment
ExecutorInfo LSWorkerHeartbeat
LSTopoHistory LSTopoHistoryList
- WorkerResources])
+ WorkerResources]
+ [org.apache.storm.utils Utils])
(:import [org.apache.storm.utils LocalState]))
(def LS-WORKER-HEARTBEAT "worker-heartbeat")
@@ -104,12 +105,14 @@
(->executor-list (.get_executors thrift-local-assignment))
(.get_resources thrift-local-assignment)))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn ls-local-assignments!
[^LocalState local-state assignments]
- (let [local-assignment-map (map-val ->LocalAssignment assignments)]
- (.put local-state LS-LOCAL-ASSIGNMENTS
+ (let [local-assignment-map (map-val ->LocalAssignment assignments)]
+ (.put local-state LS-LOCAL-ASSIGNMENTS
(LSSupervisorAssignments. local-assignment-map))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn ls-local-assignments
[^LocalState local-state]
(if-let [thrift-local-assignments (.get local-state LS-LOCAL-ASSIGNMENTS)]
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker.clj
index 2204cc487c4..c14e67eb94c 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker.clj
@@ -19,8 +19,9 @@
[java.util.concurrent ConcurrentHashMap]
[java.util.concurrent.atomic AtomicInteger]
[org.apache.storm.generated HBNodes
- HBServerMessageType HBMessage HBMessageData HBPulse]
- [org.apache.storm.utils VersionInfo ConfigUtils])
+ HBServerMessageType HBMessage HBMessageData HBPulse]
+ [org.apache.storm.utils VersionInfo ConfigUtils]
+ [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
(:use [clojure.string :only [replace-first split]]
[org.apache.storm log config util])
(:require [clojure.java.jmx :as jmx])
@@ -237,5 +238,5 @@
(PacemakerServer. (mk-handler conf) conf)))
(defn -main []
- (redirect-stdio-to-slf4j!)
+ (SysOutOverSLF4J/sendSystemOutAndErrToSLF4J)
(launch-server!))
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index cede59e0941..be4361a4b4c 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -40,6 +40,22 @@
(def max-retries 10)
+(defn retry-on-exception
+ "Retries specific function on exception based on retries count"
+ [retries task-description f & args]
+ (let [res (try {:value (apply f args)}
+ (catch Exception e
+ (if (<= 0 retries)
+ (throw e)
+ {:exception e})))]
+ (if (:exception res)
+ (do
+ (log-error (:exception res) (str "Failed to " task-description ". Will make [" retries "] more attempts."))
+ (recur (dec retries) task-description f args))
+ (do
+ (log-debug (str "Successful " task-description "."))
+ (:value res)))))
+
(defn -mkState [this conf auth-conf acls context]
(let [zk-state (makeZKState conf auth-conf acls context)
pacemaker-client (makeClient conf)]
@@ -64,7 +80,7 @@
(sync_path [this path] (.sync_path zk-state path))
(set_worker_hb [this path data acls]
- (util/retry-on-exception
+ (retry-on-exception
max-retries
"set_worker_hb"
#(let [response
@@ -79,7 +95,7 @@
(throw (HBExecutionException. "Invalid Response Type"))))))
(delete_worker_hb [this path]
- (util/retry-on-exception
+ (retry-on-exception
max-retries
"delete_worker_hb"
#(let [response
@@ -91,7 +107,7 @@
(throw (HBExecutionException. "Invalid Response Type"))))))
(get_worker_hb [this path watch?]
- (util/retry-on-exception
+ (retry-on-exception
max-retries
"get_worker_hb"
#(let [response
@@ -106,7 +122,7 @@
(throw (HBExecutionException. "Invalid Response Type"))))))
(get_worker_hb_children [this path watch?]
- (util/retry-on-exception
+ (retry-on-exception
max-retries
"get_worker_hb_children"
#(let [response
diff --git a/storm-core/src/clj/org/apache/storm/process_simulator.clj b/storm-core/src/clj/org/apache/storm/process_simulator.clj
index 03c3dd96f6f..fe5bc5b28da 100644
--- a/storm-core/src/clj/org/apache/storm/process_simulator.clj
+++ b/storm-core/src/clj/org/apache/storm/process_simulator.clj
@@ -15,9 +15,7 @@
;; limitations under the License.
(ns org.apache.storm.process-simulator
- (:use [org.apache.storm log util]))
-
-(def pid-counter (mk-counter))
+ (:use [org.apache.storm log]))
(def process-map (atom {}))
diff --git a/storm-core/src/clj/org/apache/storm/scheduler/DefaultScheduler.clj b/storm-core/src/clj/org/apache/storm/scheduler/DefaultScheduler.clj
index f6f89f8b776..71b507e97fe 100644
--- a/storm-core/src/clj/org/apache/storm/scheduler/DefaultScheduler.clj
+++ b/storm-core/src/clj/org/apache/storm/scheduler/DefaultScheduler.clj
@@ -18,14 +18,17 @@
(:require [org.apache.storm.scheduler.EvenScheduler :as EvenScheduler])
(:import [org.apache.storm.scheduler IScheduler Topologies
Cluster TopologyDetails WorkerSlot SchedulerAssignment
- EvenScheduler ExecutorDetails])
+ EvenScheduler ExecutorDetails]
+ [org.apache.storm.utils Utils])
(:gen-class
:implements [org.apache.storm.scheduler.IScheduler]))
(defn- bad-slots [existing-slots num-executors num-workers]
(if (= 0 num-workers)
'()
- (let [distribution (atom (integer-divided num-executors num-workers))
+ (let [distribution (->> (Utils/integerDivided num-executors num-workers)
+ clojurify-structure
+ atom)
keepers (atom {})]
(doseq [[node+port executor-list] existing-slots :let [executor-count (count executor-list)]]
(when (pos? (get @distribution executor-count 0))
diff --git a/storm-core/src/clj/org/apache/storm/scheduler/EvenScheduler.clj b/storm-core/src/clj/org/apache/storm/scheduler/EvenScheduler.clj
index 783da26f1df..fce535f859b 100644
--- a/storm-core/src/clj/org/apache/storm/scheduler/EvenScheduler.clj
+++ b/storm-core/src/clj/org/apache/storm/scheduler/EvenScheduler.clj
@@ -17,10 +17,21 @@
(:use [org.apache.storm util log config])
(:require [clojure.set :as set])
(:import [org.apache.storm.scheduler IScheduler Topologies
- Cluster TopologyDetails WorkerSlot ExecutorDetails])
+ Cluster TopologyDetails WorkerSlot ExecutorDetails]
+ [org.apache.storm.utils Utils])
(:gen-class
:implements [org.apache.storm.scheduler.IScheduler]))
+; this can be rewritten to be tail recursive
+(defn- interleave-all
+ [& colls]
+ (if (empty? colls)
+ []
+ (let [colls (filter (complement empty?) colls)
+ my-elems (map first colls)
+ rest-elems (apply interleave-all (map rest colls))]
+ (concat my-elems rest-elems))))
+
(defn sort-slots [all-slots]
(let [split-up (sort-by count > (vals (group-by first all-slots)))]
(apply interleave-all split-up)
@@ -35,9 +46,15 @@
:let [executor [(.getStartTask executor) (.getEndTask executor)]
node+port [(.getNodeId slot) (.getPort slot)]]]
{executor node+port}))
- alive-assigned (reverse-map executor->node+port)]
+ alive-assigned (clojurify-structure (Utils/reverseMap executor->node+port))]
alive-assigned))
+(defn- repeat-seq
+ ([aseq]
+ (apply concat (repeat aseq)))
+ ([amt aseq]
+ (apply concat (repeat amt aseq))))
+
(defn- schedule-topology [^TopologyDetails topology ^Cluster cluster]
(let [topology-id (.getId topology)
available-slots (->> (.getAvailableSlots cluster)
@@ -67,7 +84,7 @@
(doseq [^TopologyDetails topology needs-scheduling-topologies
:let [topology-id (.getId topology)
new-assignment (schedule-topology topology cluster)
- node+port->executors (reverse-map new-assignment)]]
+ node+port->executors (clojurify-structure (Utils/reverseMap new-assignment))]]
(doseq [[node+port executors] node+port->executors
:let [^WorkerSlot slot (WorkerSlot. (first node+port) (last node+port))
executors (for [[start-task end-task] executors]
diff --git a/storm-core/src/clj/org/apache/storm/scheduler/IsolationScheduler.clj b/storm-core/src/clj/org/apache/storm/scheduler/IsolationScheduler.clj
index 2e867484cbd..151fcbb2b69 100644
--- a/storm-core/src/clj/org/apache/storm/scheduler/IsolationScheduler.clj
+++ b/storm-core/src/clj/org/apache/storm/scheduler/IsolationScheduler.clj
@@ -16,10 +16,13 @@
(ns org.apache.storm.scheduler.IsolationScheduler
(:use [org.apache.storm util config log])
(:require [org.apache.storm.scheduler.DefaultScheduler :as DefaultScheduler])
- (:import [java.util HashSet Set List LinkedList ArrayList Map HashMap])
+ (:import [java.util HashSet Set List LinkedList ArrayList Map HashMap]
+ [org.apache.storm.utils])
+ (:import [org.apache.storm.utils Utils Container])
(:import [org.apache.storm.scheduler IScheduler Topologies
Cluster TopologyDetails WorkerSlot SchedulerAssignment
- EvenScheduler ExecutorDetails])
+ EvenScheduler ExecutorDetails]
+ [org.apache.storm.utils Utils])
(:gen-class
:init init
:constructors {[] []}
@@ -27,15 +30,23 @@
:implements [org.apache.storm.scheduler.IScheduler]))
(defn -init []
- [[] (container)])
+ [[] (Container.)])
(defn -prepare [this conf]
- (container-set! (.state this) conf))
+ (.. this state (set conf)))
+(defn- repeat-seq
+ ([aseq]
+ (apply concat (repeat aseq)))
+ ([amt aseq]
+ (apply concat (repeat amt aseq))))
+
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- compute-worker-specs "Returns mutable set of sets of executors"
[^TopologyDetails details]
(->> (.getExecutorToComponent details)
- reverse-map
+ (Utils/reverseMap)
+ clojurify-structure
(map second)
(apply concat)
(map vector (repeat-seq (range (.getNumWorkers details))))
@@ -61,7 +72,8 @@
(let [name->machines (get conf ISOLATION-SCHEDULER-MACHINES)
machines (get name->machines (.getName topology))
workers (.getNumWorkers topology)]
- (-> (integer-divided workers machines)
+ (-> (Utils/integerDivided workers machines)
+ clojurify-structure
(dissoc 0)
(HashMap.)
)))
@@ -75,7 +87,8 @@
(letfn [(to-slot-specs [^SchedulerAssignment ass]
(->> ass
.getExecutorToSlot
- reverse-map
+ (Utils/reverseMap)
+ clojurify-structure
(map (fn [[slot executors]]
[slot (.getTopologyId ass) (set executors)]))))]
(->> cluster
@@ -155,7 +168,7 @@
;; run default scheduler on isolated topologies that didn't have enough slots + non-isolated topologies on remaining machines
;; set blacklist to what it was initially
(defn -schedule [this ^Topologies topologies ^Cluster cluster]
- (let [conf (container-get (.state this))
+ (let [conf (.. this state (get))
orig-blacklist (HashSet. (.getBlacklistedHosts cluster))
iso-topologies (isolated-topologies conf (.getTopologies topologies))
iso-ids-set (->> iso-topologies (map #(.getId ^TopologyDetails %)) set)
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
index 68b16fd2f07..4f25f539c0a 100644
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@ -24,7 +24,8 @@
ExecutorAggregateStats SpecificAggregateStats
SpoutAggregateStats TopologyPageInfo TopologyStats])
(:import [org.apache.storm.utils Utils])
- (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
+ (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric]
+ [java.util Collection])
(:use [org.apache.storm log util])
(:use [clojure.math.numeric-tower :only [ceil]]))
@@ -53,6 +54,11 @@
(def NUM-STAT-BUCKETS 20)
+(defn- div
+ "Perform floating point division on the arguments."
+ [f & rest]
+ (apply / (double f) rest))
+
(defn- mk-common-stats
[rate]
(CommonStats.
@@ -200,6 +206,10 @@
(value-stats stats SPOUT-FIELDS)
{:type :spout}))
+(defn- class-selector
+ [obj & args]
+ (class obj))
+
(defmulti render-stats! class-selector)
(defmethod render-stats! SpoutExecutorStats
@@ -324,17 +334,17 @@
(letfn [(weight-avg [[id avg]]
(let [num-e (get idk->num-executed id)]
(product-or-0 avg num-e)))]
- {:executeLatencyTotal (sum (map weight-avg idk->exec-avg))
- :processLatencyTotal (sum (map weight-avg idk->proc-avg))
- :executed (sum (vals idk->num-executed))}))
+ {:executeLatencyTotal (reduce + (map weight-avg idk->exec-avg))
+ :processLatencyTotal (reduce + (map weight-avg idk->proc-avg))
+ :executed (reduce + (vals idk->num-executed))}))
(defn- agg-spout-lat-and-count
"Aggregates number acked and complete latencies across all streams."
[sid->comp-avg sid->num-acked]
(letfn [(weight-avg [[id avg]]
(product-or-0 avg (get sid->num-acked id)))]
- {:completeLatencyTotal (sum (map weight-avg sid->comp-avg))
- :acked (sum (vals sid->num-acked))}))
+ {:completeLatencyTotal (reduce + (map weight-avg sid->comp-avg))
+ :acked (reduce + (vals sid->num-acked))}))
(defn add-pairs
([] [0 0])
@@ -347,6 +357,7 @@
(fn [_] true)
(fn [stream] (and (string? stream) (not (Utils/isSystemId stream))))))
+;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
(defn mk-include-sys-filter
"Returns a function that includes or excludes map entries whose keys are
system ids."
@@ -421,6 +432,7 @@
statk->w->sid->num :stats}
window
include-sys?]
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(let [str-key (partial map-key str)
handle-sys-components-fn (mk-include-sys-filter include-sys?)]
{:executor-id exec-id,
@@ -477,6 +489,7 @@
statk->w->sid->num :stats}
window
include-sys?]
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(let [str-key (partial map-key str)
handle-sys-components-fn (mk-include-sys-filter include-sys?)]
{:executor-id exec-id,
@@ -523,6 +536,7 @@
uptime :uptime}
window
include-sys?]
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(let [str-key (partial map-key str)
handle-sys-components-fn (mk-include-sys-filter include-sys?)]
{comp-id
@@ -547,27 +561,27 @@
(get window)
handle-sys-components-fn
vals
- sum)
+ (reduce +))
:transferred (-> statk->w->sid->num
:transferred
str-key
(get window)
handle-sys-components-fn
vals
- sum)
+ (reduce +))
:capacity (compute-agg-capacity statk->w->sid->num uptime)
:acked (-> statk->w->sid->num
:acked
str-key
(get window)
vals
- sum)
+ (reduce +))
:failed (-> statk->w->sid->num
:failed
str-key
(get window)
vals
- sum)})}))
+ (reduce +))})}))
(defn agg-pre-merge-topo-page-spout
[{comp-id :comp-id
@@ -575,6 +589,7 @@
statk->w->sid->num :stats}
window
include-sys?]
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(let [str-key (partial map-key str)
handle-sys-components-fn (mk-include-sys-filter include-sys?)]
{comp-id
@@ -595,20 +610,20 @@
(get window)
handle-sys-components-fn
vals
- sum)
+ (reduce +))
:transferred (-> statk->w->sid->num
:transferred
str-key
(get window)
handle-sys-components-fn
vals
- sum)
+ (reduce +))
:failed (-> statk->w->sid->num
:failed
str-key
(get window)
vals
- sum)})}))
+ (reduce +))})}))
(defn merge-agg-comp-stats-comp-page-bolt
[{acc-in :cid+sid->input-stats
@@ -702,11 +717,13 @@
:acked (sum-or-0 (:acked acc-spout-stats) (:acked spout-stats))
:failed (sum-or-0 (:failed acc-spout-stats) (:failed spout-stats))})
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn aggregate-count-streams
[stats]
(->> stats
- (map-val #(reduce + (vals %)))))
+ (map-val #(reduce + (vals %)))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- agg-topo-exec-stats*
"A helper function that does the common work to aggregate stats of one
executor with the given map for the topology page."
@@ -896,13 +913,17 @@
0))
(dissoc :completeLatencyTotal)
(assoc :lastError (last-err-fn id)))]))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->emitted (map-key str (:window->emitted acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->transferred (map-key str (:window->transferred acc-data))
:window->complete-latency
(compute-weighted-averages-per-window acc-data
:window->comp-lat-wgt-avg
:window->acked)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->acked (map-key str (:window->acked acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->failed (map-key str (:window->failed acc-data))})
(defn- thriftify-common-agg-stats
@@ -1017,6 +1038,7 @@
(post-aggregate-topo-stats task->component exec->node+port last-err-fn)
(thriftify-topo-page-data topology-id)))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- agg-bolt-exec-win-stats
"A helper function that aggregates windowed stats from one bolt executor."
[acc-stats new-stats include-sys?]
@@ -1052,6 +1074,7 @@
aggregate-count-streams
(merge-with + (:window->failed acc-stats)))}))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- agg-spout-exec-win-stats
"A helper function that aggregates windowed stats from one spout executor."
[acc-stats new-stats include-sys?]
@@ -1144,6 +1167,7 @@
(defmulti post-aggregate-comp-stats
(fn [_ _ data] (:type data)))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defmethod post-aggregate-comp-stats :bolt
[task->component
exec->host+port
@@ -1172,20 +1196,26 @@
:processLatencyTotal))))))
:sid->output-stats o-stats
:executor-stats (:executor-stats (:stats acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->emitted (map-key str (:window->emitted acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->transferred (map-key str (:window->transferred acc-data))
:window->execute-latency
(compute-weighted-averages-per-window acc-data
:window->exec-lat-wgt-avg
:window->executed)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->executed (map-key str (:window->executed acc-data))
:window->process-latency
(compute-weighted-averages-per-window acc-data
:window->proc-lat-wgt-avg
:window->executed)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->acked (map-key str (:window->acked acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->failed (map-key str (:window->failed acc-data))})
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defmethod post-aggregate-comp-stats :spout
[task->component
exec->host+port
@@ -1206,13 +1236,17 @@
{:complete-latency 0})]
(-> m (merge lat) (dissoc :completeLatencyTotal))))))
:executor-stats (:executor-stats (:stats acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->emitted (map-key str (:window->emitted acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->transferred (map-key str (:window->transferred acc-data))
:window->complete-latency
(compute-weighted-averages-per-window acc-data
:window->comp-lat-wgt-avg
:window->acked)
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->acked (map-key str (:window->acked acc-data))
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
:window->failed (map-key str (:window->failed acc-data))})
(defmethod post-aggregate-comp-stats :default [& _] {})
@@ -1236,14 +1270,17 @@
[(to-global-stream-id cid+sid)
(thriftify-bolt-agg-stats input-stats)])))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- thriftify-bolt-output-stats
[sid->output-stats]
(map-val thriftify-bolt-agg-stats sid->output-stats))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn- thriftify-spout-output-stats
[sid->output-stats]
(map-val thriftify-spout-agg-stats sid->output-stats))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn thriftify-comp-page-data
[topo-id topology comp-id data]
(let [w->stats (swap-map-order
@@ -1336,6 +1373,7 @@
(if (= c 0) 0
(double (/ t c))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn aggregate-averages
[average-seq counts-seq]
(->> (expand-averages-seq average-seq counts-seq)
@@ -1343,6 +1381,7 @@
(fn [s]
(map-val val-avg s)))))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn aggregate-avg-streams
[avg counts]
(let [expanded (expand-averages avg counts)]
@@ -1350,6 +1389,7 @@
(map-val #(reduce add-pairs (vals %)))
(map-val val-avg))))
+;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
(defn pre-process
[stream-summary include-sys?]
(let [filter-fn (mk-include-sys-fn include-sys?)
@@ -1376,6 +1416,12 @@
{:emitted (aggregate-counts (map #(.get_emitted ^ExecutorStats %) stats-seq))
:transferred (aggregate-counts (map #(.get_transferred ^ExecutorStats %) stats-seq))})
+(defn- collectify
+ [obj]
+ (if (or (sequential? obj) (instance? Collection obj))
+ obj
+ [obj]))
+
(defn aggregate-bolt-stats
[stats-seq include-sys?]
(let [stats-seq (collectify stats-seq)]
@@ -1459,10 +1505,10 @@
(aggregate-bolt-streams)
swap-map-order
(get (str TEN-MIN-IN-SECONDS))))
- uptime (nil-to-zero (.get_uptime_secs e))
+ uptime (Utils/nullToZero (.get_uptime_secs e))
window (if (< uptime TEN-MIN-IN-SECONDS) uptime TEN-MIN-IN-SECONDS)
- executed (-> stats :executed nil-to-zero)
- latency (-> stats :execute-latencies nil-to-zero)]
+ executed (-> stats :executed Utils/nullToZero)
+ latency (-> stats :execute-latencies Utils/nullToZero)]
(if (> window 0)
(div (* executed latency) (* 1000 window)))))
@@ -1517,5 +1563,5 @@
[executors]
(->> executors
(map compute-executor-capacity)
- (map nil-to-zero)
+ (map #(Utils/nullToZero %))
(apply max)))
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index cc786590e87..9a487af8d96 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -23,12 +23,13 @@
[executor :as executor]])
(:require [org.apache.storm [process-simulator :as psim]])
(:import [org.apache.commons.io FileUtils]
+ [org.apache.storm.utils]
[org.apache.storm.zookeeper Zookeeper])
(:import [java.io File])
(:import [java.util HashMap ArrayList])
(:import [java.util.concurrent.atomic AtomicInteger])
(:import [java.util.concurrent ConcurrentHashMap])
- (:import [org.apache.storm.utils Time Utils RegisteredGlobalState ConfigUtils])
+ (:import [org.apache.storm.utils Time Utils IPredicate RegisteredGlobalState ConfigUtils])
(:import [org.apache.storm.tuple Fields Tuple TupleImpl])
(:import [org.apache.storm.task TopologyContext])
(:import [org.apache.storm.generated GlobalStreamId Bolt KillOptions])
@@ -44,7 +45,8 @@
(:import [org.apache.storm.transactional.partitioned PartitionedTransactionalSpoutExecutor])
(:import [org.apache.storm.tuple Tuple])
(:import [org.apache.storm.generated StormTopology])
- (:import [org.apache.storm.task TopologyContext])
+ (:import [org.apache.storm.task TopologyContext]
+ [org.json.simple JSONValue])
(:require [org.apache.storm [zookeeper :as zk]])
(:require [org.apache.storm.messaging.loader :as msg-loader])
(:require [org.apache.storm.daemon.acker :as acker])
@@ -56,7 +58,7 @@
(defn local-temp-path
[]
- (str (System/getProperty "java.io.tmpdir") (if-not on-windows? "/") (uuid)))
+ (str (System/getProperty "java.io.tmpdir") (if-not (Utils/isOnWindows) "/") (Utils/uuid)))
(defn delete-all
[paths]
@@ -99,6 +101,29 @@
(defn advance-time-secs! [secs]
(advance-time-ms! (* (long secs) 1000)))
+(defn set-var-root*
+ [avar val]
+ (alter-var-root avar (fn [avar] val)))
+
+(defmacro set-var-root
+ [var-sym val]
+ `(set-var-root* (var ~var-sym) ~val))
+
+(defmacro with-var-roots
+ [bindings & body]
+ (let [settings (partition 2 bindings)
+ tmpvars (repeatedly (count settings) (partial gensym "old"))
+ vars (map first settings)
+ savevals (vec (mapcat (fn [t v] [t v]) tmpvars vars))
+ setters (for [[v s] settings] `(set-var-root ~v ~s))
+ restorers (map (fn [v s] `(set-var-root ~v ~s)) vars tmpvars)]
+ `(let ~savevals
+ ~@setters
+ (try
+ ~@body
+ (finally
+ ~@restorers)))))
+
(defnk add-supervisor
[cluster-map :ports 2 :conf {} :id nil]
(let [tmp-dir (local-temp-path)
@@ -128,6 +153,12 @@
server))
+(defn- mk-counter
+ ([] (mk-counter 1))
+ ([start-val]
+ (let [val (atom (dec start-val))]
+ (fn [] (swap! val inc)))))
+
;; returns map containing cluster info
;; local dir is always overridden in maps
;; can customize the supervisors (except for ports) by passing in map for :supervisors parameter
@@ -173,14 +204,22 @@
cluster-map))
(defn get-supervisor [cluster-map supervisor-id]
- (let [finder-fn #(= (.get-id %) supervisor-id)]
- (find-first finder-fn @(:supervisors cluster-map))))
+ (let [pred (reify IPredicate (test [this x] (= (.get-id x) supervisor-id)))]
+ (Utils/findOne pred @(:supervisors cluster-map))))
+
+(defn remove-first
+ [pred aseq]
+ (let [[b e] (split-with (complement pred) aseq)]
+ (when (empty? e)
+ (throw (IllegalArgumentException. "Nothing to remove")))
+ (concat b (rest e))))
(defn kill-supervisor [cluster-map supervisor-id]
(let [finder-fn #(= (.get-id %) supervisor-id)
+ pred (reify IPredicate (test [this x] (= (.get-id x) supervisor-id)))
supervisors @(:supervisors cluster-map)
- sup (find-first finder-fn
- supervisors)]
+ sup (Utils/findOne pred
+ supervisors)]
;; tmp-dir will be taken care of by shutdown
(reset! (:supervisors cluster-map) (remove-first finder-fn supervisors))
(.shutdown sup)))
@@ -209,13 +248,13 @@
(doseq [t @(:tmp-dirs cluster-map)]
(log-message "Deleting temporary path " t)
(try
- (rmr t)
+ (Utils/forceDelete t)
;; on windows, the host process still holds lock on the logfile
(catch Exception e (log-message (.getMessage e)))) ))
(def TEST-TIMEOUT-MS
(let [timeout (System/getenv "STORM_TEST_TIMEOUT_MS")]
- (parse-int (if timeout timeout "5000"))))
+ (Integer/parseInt (if timeout timeout "5000"))))
(defmacro while-timeout [timeout-ms condition & body]
`(let [end-time# (+ (System/currentTimeMillis) ~timeout-ms)]
@@ -299,13 +338,13 @@
[nimbus storm-name conf topology]
(when-not (Utils/isValidConf conf)
(throw (IllegalArgumentException. "Topology conf is not json-serializable")))
- (.submitTopology nimbus storm-name nil (to-json conf) topology))
+ (.submitTopology nimbus storm-name nil (JSONValue/toJSONString conf) topology))
(defn submit-local-topology-with-opts
[nimbus storm-name conf topology submit-opts]
(when-not (Utils/isValidConf conf)
(throw (IllegalArgumentException. "Topology conf is not json-serializable")))
- (.submitTopologyWithOpts nimbus storm-name nil (to-json conf) topology submit-opts))
+ (.submitTopologyWithOpts nimbus storm-name nil (JSONValue/toJSONString conf) topology submit-opts))
(defn mocked-convert-assignments-to-worker->resources [storm-cluster-state storm-name worker->resources]
(fn [existing-assignments]
@@ -353,7 +392,7 @@
[supervisor-conf port]
(let [supervisor-state (ConfigUtils/supervisorState supervisor-conf)
worker->port (ls-approved-workers supervisor-state)]
- (first ((reverse-map worker->port) port))))
+ (first ((clojurify-structure (Utils/reverseMap worker->port)) port))))
(defn find-worker-port
[supervisor-conf worker-id]
@@ -395,10 +434,13 @@
(let [state (:storm-cluster-state cluster-map)
nimbus (:nimbus cluster-map)
storm-id (common/get-storm-id state storm-name)
- component->tasks (reverse-map
+ component->tasks (clojurify-structure (Utils/reverseMap
(common/storm-task-info
(.getUserTopology nimbus storm-id)
- (from-json (.getTopologyConf nimbus storm-id))))
+ (->>
+ (.getTopologyConf nimbus storm-id)
+ (#(if % (JSONValue/parse %)))
+ clojurify-structure))))
component->tasks (if component-ids
(select-keys component->tasks component-ids)
component->tasks)
@@ -497,7 +539,7 @@
capturer (TupleCaptureBolt.)]
(.set_bolts topology
(assoc (clojurify-structure bolts)
- (uuid)
+ (Utils/uuid)
(Bolt.
(serialize-component-object capturer)
(mk-plain-component-common (into {} (for [[id direct?] all-streams]
@@ -510,6 +552,7 @@
:capturer capturer}))
;; TODO: mock-sources needs to be able to mock out state spouts as well
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defnk complete-topology
[cluster-map topology
:mock-sources {}
@@ -520,7 +563,7 @@
;; TODO: the idea of mocking for transactional topologies should be done an
;; abstraction level above... should have a complete-transactional-topology for this
(let [{topology :topology capturer :capturer} (capture-topology topology)
- storm-name (or topology-name (str "topologytest-" (uuid)))
+ storm-name (or topology-name (str "topologytest-" (Utils/uuid)))
state (:storm-cluster-state cluster-map)
spouts (.get_spouts topology)
replacements (map-val (fn [v]
@@ -573,6 +616,12 @@
([results component-id]
(read-tuples results component-id Utils/DEFAULT_STREAM_ID)))
+(defn multi-set
+ "Returns a map of elem to count"
+ [aseq]
+ (apply merge-with +
+ (map #(hash-map % 1) aseq)))
+
(defn ms=
[& args]
(apply = (map multi-set args)))
@@ -614,7 +663,7 @@
(defmacro with-tracked-cluster
[[cluster-sym & cluster-args] & body]
- `(let [id# (uuid)]
+ `(let [id# (Utils/uuid)]
(RegisteredGlobalState/setState
id#
(doto (ConcurrentHashMap.)
diff --git a/storm-core/src/clj/org/apache/storm/thrift.clj b/storm-core/src/clj/org/apache/storm/thrift.clj
index b5af521010a..779c1d1848b 100644
--- a/storm-core/src/clj/org/apache/storm/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/thrift.clj
@@ -29,7 +29,8 @@
(:import [org.apache.storm.grouping CustomStreamGrouping])
(:import [org.apache.storm.topology TopologyBuilder])
(:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
- (:import [org.apache.thrift.transport TTransport])
+ (:import [org.apache.thrift.transport TTransport]
+ (org.json.simple JSONValue))
(:use [org.apache.storm util config log zookeeper]))
(defn instantiate-java-object
@@ -107,6 +108,7 @@
[fields]
(StreamInfo. fields false))
+;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
(defn mk-output-spec
[output-spec]
(let [output-spec (if (map? output-spec)
@@ -125,7 +127,7 @@
(when parallelism-hint
(.set_parallelism_hint ret parallelism-hint))
(when conf
- (.set_json_conf ret (to-json conf)))
+ (.set_json_conf ret (JSONValue/toJSONString conf)))
ret))
(defnk mk-spout-spec*
diff --git a/storm-core/src/clj/org/apache/storm/timer.clj b/storm-core/src/clj/org/apache/storm/timer.clj
index 0d8839e6f93..5f31032c3a6 100644
--- a/storm-core/src/clj/org/apache/storm/timer.clj
+++ b/storm-core/src/clj/org/apache/storm/timer.clj
@@ -15,7 +15,7 @@
;; limitations under the License.
(ns org.apache.storm.timer
- (:import [org.apache.storm.utils Time])
+ (:import [org.apache.storm.utils Utils Time])
(:import [java.util PriorityQueue Comparator Random])
(:import [java.util.concurrent Semaphore])
(:use [org.apache.storm util log]))
@@ -41,7 +41,7 @@
(while @active
(try
(let [[time-millis _ _ :as elem] (locking lock (.peek queue))]
- (if (and elem (>= (current-time-millis) time-millis))
+ (if (and elem (>= (Time/currentTimeMillis) time-millis))
;; It is imperative to not run the function
;; inside the timer lock. Otherwise, it is
;; possible to deadlock if the fn deals with
@@ -57,7 +57,7 @@
;; an upper bound, e.g. 1000 millis, to the
;; sleeping time, to limit the response time
;; for detecting any new event within 1 secs.
- (Time/sleep (min 1000 (- time-millis (current-time-millis))))
+ (Time/sleep (min 1000 (- time-millis (Time/currentTimeMillis))))
;; Otherwise poll to see if any new event
;; was scheduled. This is, in essence, the
;; response time for detecting any new event
@@ -67,7 +67,7 @@
(catch Throwable t
;; Because the interrupted exception can be
;; wrapped in a RuntimeException.
- (when-not (exception-cause? InterruptedException t)
+ (when-not (Utils/exceptionCauseIsInstanceOf InterruptedException t)
(kill-fn t)
(reset! active false)
(throw t)))))
@@ -90,9 +90,9 @@
(defnk schedule
[timer delay-secs afn :check-active true :jitter-ms 0]
(when check-active (check-active! timer))
- (let [id (uuid)
+ (let [id (Utils/uuid)
^PriorityQueue queue (:queue timer)
- end-time-ms (+ (current-time-millis) (secs-to-millis-long delay-secs))
+ end-time-ms (+ (Time/currentTimeMillis) (Time/secsToMillisLong delay-secs))
end-time-ms (if (< 0 jitter-ms) (+ (.nextInt (:random timer) jitter-ms) end-time-ms) end-time-ms)]
(locking (:lock timer)
(.add queue [end-time-ms afn id]))))
diff --git a/storm-core/src/clj/org/apache/storm/trident/testing.clj b/storm-core/src/clj/org/apache/storm/trident/testing.clj
index 44e5ca9e2d2..0ec5613b095 100644
--- a/storm-core/src/clj/org/apache/storm/trident/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/trident/testing.clj
@@ -19,7 +19,8 @@
(:require [org.apache.storm [LocalDRPC]])
(:import [org.apache.storm LocalDRPC])
(:import [org.apache.storm.tuple Fields])
- (:import [org.apache.storm.generated KillOptions])
+ (:import [org.apache.storm.generated KillOptions]
+ [org.json.simple JSONValue])
(:require [org.apache.storm [testing :as t]])
(:use [org.apache.storm util])
)
@@ -28,11 +29,11 @@
(LocalDRPC.))
(defn exec-drpc [^LocalDRPC drpc function-name args]
- (let [res (.execute drpc function-name args)]
- (from-json res)))
+ (if-let [res (.execute drpc function-name args)]
+ (clojurify-structure (JSONValue/parse res))))
(defn exec-drpc-tuples [^LocalDRPC drpc function-name tuples]
- (exec-drpc drpc function-name (to-json tuples)))
+ (exec-drpc drpc function-name (JSONValue/toJSONString tuples)))
(defn feeder-spout [fields]
(FeederBatchSpout. fields))
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index 220925459e6..1bf85d44387 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -26,7 +26,7 @@
(:use [org.apache.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
ACKER-FAIL-STREAM-ID mk-authorization-handler
start-metrics-reporters]]])
- (:import [org.apache.storm.utils Utils]
+ (:import [org.apache.storm.utils Time]
[org.apache.storm.generated NimbusSummary])
(:use [clojure.string :only [blank? lower-case trim split]])
(:import [org.apache.storm.generated ExecutorSpecificStats
@@ -41,9 +41,11 @@
(:import [org.apache.storm.security.auth AuthUtils ReqContext])
(:import [org.apache.storm.generated AuthorizationException ProfileRequest ProfileAction NodeInfo])
(:import [org.apache.storm.security.auth AuthUtils])
- (:import [org.apache.storm.utils VersionInfo ConfigUtils])
+ (:import [org.apache.storm.utils Utils VersionInfo ConfigUtils])
(:import [org.apache.storm Config])
(:import [java.io File])
+ (:import [java.net URLEncoder URLDecoder])
+ (:import [org.json.simple JSONValue])
(:require [compojure.route :as route]
[compojure.handler :as handler]
[ring.util.response :as resp]
@@ -143,10 +145,10 @@
(defn event-log-link
[topology-id component-id host port secure?]
- (logviewer-link host (event-logs-filename topology-id port) secure?))
+ (logviewer-link host (Utils/eventLogsFilename topology-id port) secure?))
(defn worker-log-link [host port topology-id secure?]
- (let [fname (logs-filename topology-id port)]
+ (let [fname (Utils/logsFilename topology-id port)]
(logviewer-link host fname secure?)))
(defn nimbus-log-link [host]
@@ -158,7 +160,7 @@
(defn get-error-time
[error]
(if error
- (time-delta (.get_error_time_secs ^ErrorInfo error))))
+ (Time/deltaSecs (.get_error_time_secs ^ErrorInfo error))))
(defn get-error-data
[error]
@@ -186,10 +188,10 @@
(defn worker-dump-link [host port topology-id]
(url-format "http://%s:%s/dumps/%s/%s"
- (url-encode host)
+ (URLEncoder/encode host)
(*STORM-CONF* LOGVIEWER-PORT)
- (url-encode topology-id)
- (str (url-encode host) ":" (url-encode port))))
+ (URLEncoder/encode topology-id)
+ (str (URLEncoder/encode host) ":" (URLEncoder/encode port))))
(defn stats-times
[stats-map]
@@ -303,6 +305,7 @@
bolt-summs (filter (partial bolt-summary? topology) execs)
spout-comp-summs (group-by-comp spout-summs)
bolt-comp-summs (group-by-comp bolt-summs)
+ ;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
bolt-comp-summs (filter-key (mk-include-sys-fn include-sys?)
bolt-comp-summs)]
(visualization-data
@@ -310,6 +313,13 @@
(hashmap-to-persistent bolts))
spout-comp-summs bolt-comp-summs window id))))
+(defn- from-json
+ [^String str]
+ (if str
+ (clojurify-structure
+ (JSONValue/parse str))
+ nil))
+
(defn validate-tplg-submit-params [params]
(let [tplg-jar-file (params :topologyJar)
tplg-config (if (not-nil? (params :topologyConfig)) (from-json (params :topologyConfig)))]
@@ -323,12 +333,12 @@
(let [tplg-main-class (if (not-nil? tplg-config) (trim (tplg-config "topologyMainClass")))
tplg-main-class-args (if (not-nil? tplg-config) (tplg-config "topologyMainClassArgs"))
storm-home (System/getProperty "storm.home")
- storm-conf-dir (str storm-home file-path-separator "conf")
+ storm-conf-dir (str storm-home Utils/FILE_PATH_SEPARATOR "conf")
storm-log-dir (if (not-nil? (*STORM-CONF* "storm.log.dir")) (*STORM-CONF* "storm.log.dir")
- (str storm-home file-path-separator "logs"))
- storm-libs (str storm-home file-path-separator "lib" file-path-separator "*")
- java-cmd (str (System/getProperty "java.home") file-path-separator "bin" file-path-separator "java")
- storm-cmd (str storm-home file-path-separator "bin" file-path-separator "storm")
+ (str storm-home Utils/FILE_PATH_SEPARATOR "logs"))
+ storm-libs (str storm-home Utils/FILE_PATH_SEPARATOR "lib" Utils/FILE_PATH_SEPARATOR "*")
+ java-cmd (str (System/getProperty "java.home") Utils/FILE_PATH_SEPARATOR "bin" Utils/FILE_PATH_SEPARATOR "java")
+ storm-cmd (str storm-home Utils/FILE_PATH_SEPARATOR "bin" Utils/FILE_PATH_SEPARATOR "storm")
tplg-cmd-response (apply sh
(flatten
[storm-cmd "jar" tplg-jar-file tplg-main-class
@@ -449,7 +459,7 @@
(for [^TopologySummary t summs]
{
"id" (.get_id t)
- "encodedId" (url-encode (.get_id t))
+ "encodedId" (URLEncoder/encode (.get_id t))
"owner" (.get_owner t)
"name" (.get_name t)
"status" (.get_status t)
@@ -497,6 +507,7 @@
bolt-executor-summaries (filter (partial bolt-summary? storm-topology) (.get_executors topology-info))
spout-comp-id->executor-summaries (group-by-comp spout-executor-summaries)
bolt-comp-id->executor-summaries (group-by-comp bolt-executor-summaries)
+ ;TODO: when translating this function, you should replace the filter-val with a proper for loop + if condition HERE
bolt-comp-id->executor-summaries (filter-key (mk-include-sys-fn include-sys?) bolt-comp-id->executor-summaries)
id->spout-spec (.get_spouts storm-topology)
id->bolt (.get_bolts storm-topology)
@@ -541,7 +552,7 @@
(common-agg-stats-json cs)
(get-error-json topo-id (.get_last_error s) secure?)
{"spoutId" id
- "encodedSpoutId" (url-encode id)
+ "encodedSpoutId" (URLEncoder/encode id)
"completeLatency" (float-str (.get_complete_latency_ms ss))})))
(defmethod comp-agg-stats-json ComponentType/BOLT
@@ -552,7 +563,7 @@
(common-agg-stats-json cs)
(get-error-json topo-id (.get_last_error s) secure?)
{"boltId" id
- "encodedBoltId" (url-encode id)
+ "encodedBoltId" (URLEncoder/encode id)
"capacity" (float-str (.get_capacity ss))
"executeLatency" (float-str (.get_execute_latency_ms ss))
"executed" (.get_executed ss)
@@ -576,7 +587,7 @@
(.get_samplingpct debug-opts)])
uptime (.get_uptime_secs topo-info)]
{"id" id
- "encodedId" (url-encode id)
+ "encodedId" (URLEncoder/encode id)
"owner" (.get_owner topo-info)
"name" (.get_name topo-info)
"status" (.get_status topo-info)
@@ -691,13 +702,13 @@
^CommonAggregateStats cas (.get_common_stats stats)
comp-id (.get_componentId s)]
{"component" comp-id
- "encodedComponentId" (url-encode comp-id)
+ "encodedComponentId" (URLEncoder/encode comp-id)
"stream" (.get_streamId s)
"executeLatency" (float-str (.get_execute_latency_ms bas))
"processLatency" (float-str (.get_process_latency_ms bas))
- "executed" (nil-to-zero (.get_executed bas))
- "acked" (nil-to-zero (.get_acked cas))
- "failed" (nil-to-zero (.get_failed cas))}))
+ "executed" (Utils/nullToZero (.get_executed bas))
+ "acked" (Utils/nullToZero (.get_acked cas))
+ "failed" (Utils/nullToZero (.get_failed cas))}))
(defmulti unpack-comp-output-stat
(fn [[_ ^ComponentAggregateStats s]] (.get_type s)))
@@ -706,8 +717,8 @@
[[stream-id ^ComponentAggregateStats stats]]
(let [^CommonAggregateStats cas (.get_common_stats stats)]
{"stream" stream-id
- "emitted" (nil-to-zero (.get_emitted cas))
- "transferred" (nil-to-zero (.get_transferred cas))}))
+ "emitted" (Utils/nullToZero (.get_emitted cas))
+ "transferred" (Utils/nullToZero (.get_transferred cas))}))
(defmethod unpack-comp-output-stat ComponentType/SPOUT
[[stream-id ^ComponentAggregateStats stats]]
@@ -715,11 +726,11 @@
^SpecificAggregateStats spec-s (.get_specific_stats stats)
^SpoutAggregateStats spout-s (.get_spout spec-s)]
{"stream" stream-id
- "emitted" (nil-to-zero (.get_emitted cas))
- "transferred" (nil-to-zero (.get_transferred cas))
+ "emitted" (Utils/nullToZero (.get_emitted cas))
+ "transferred" (Utils/nullToZero (.get_transferred cas))
"completeLatency" (float-str (.get_complete_latency_ms spout-s))
- "acked" (nil-to-zero (.get_acked cas))
- "failed" (nil-to-zero (.get_failed cas))}))
+ "acked" (Utils/nullToZero (.get_acked cas))
+ "failed" (Utils/nullToZero (.get_failed cas))}))
(defmulti unpack-comp-exec-stat
(fn [_ _ ^ComponentAggregateStats cas] (.get_type (.get_stats ^ExecutorAggregateStats cas))))
@@ -737,19 +748,19 @@
exec-id (pretty-executor-info info)
uptime (.get_uptime_secs summ)]
{"id" exec-id
- "encodedId" (url-encode exec-id)
+ "encodedId" (URLEncoder/encode exec-id)
"uptime" (pretty-uptime-sec uptime)
"uptimeSeconds" uptime
"host" host
"port" port
- "emitted" (nil-to-zero (.get_emitted cas))
- "transferred" (nil-to-zero (.get_transferred cas))
- "capacity" (float-str (nil-to-zero (.get_capacity bas)))
+ "emitted" (Utils/nullToZero (.get_emitted cas))
+ "transferred" (Utils/nullToZero (.get_transferred cas))
+ "capacity" (float-str (Utils/nullToZero (.get_capacity bas)))
"executeLatency" (float-str (.get_execute_latency_ms bas))
- "executed" (nil-to-zero (.get_executed bas))
+ "executed" (Utils/nullToZero (.get_executed bas))
"processLatency" (float-str (.get_process_latency_ms bas))
- "acked" (nil-to-zero (.get_acked cas))
- "failed" (nil-to-zero (.get_failed cas))
+ "acked" (Utils/nullToZero (.get_acked cas))
+ "failed" (Utils/nullToZero (.get_failed cas))
"workerLogLink" (worker-log-link host port topology-id secure?)}))
(defmethod unpack-comp-exec-stat ComponentType/SPOUT
@@ -765,16 +776,16 @@
exec-id (pretty-executor-info info)
uptime (.get_uptime_secs summ)]
{"id" exec-id
- "encodedId" (url-encode exec-id)
+ "encodedId" (URLEncoder/encode exec-id)
"uptime" (pretty-uptime-sec uptime)
"uptimeSeconds" uptime
"host" host
"port" port
- "emitted" (nil-to-zero (.get_emitted cas))
- "transferred" (nil-to-zero (.get_transferred cas))
+ "emitted" (Utils/nullToZero (.get_emitted cas))
+ "transferred" (Utils/nullToZero (.get_transferred cas))
"completeLatency" (float-str (.get_complete_latency_ms sas))
- "acked" (nil-to-zero (.get_acked cas))
- "failed" (nil-to-zero (.get_failed cas))
+ "acked" (Utils/nullToZero (.get_acked cas))
+ "failed" (Utils/nullToZero (.get_failed cas))
"workerLogLink" (worker-log-link host port topology-id secure?)}))
(defmulti unpack-component-page-info
@@ -842,13 +853,13 @@
secure?)
"user" user
"id" component
- "encodedId" (url-encode component)
+ "encodedId" (URLEncoder/encode component)
"name" (.get_topology_name comp-page-info)
"executors" (.get_num_executors comp-page-info)
"tasks" (.get_num_tasks comp-page-info)
"topologyId" topology-id
"topologyStatus" (.get_topology_status comp-page-info)
- "encodedTopologyId" (url-encode topology-id)
+ "encodedTopologyId" (URLEncoder/encode topology-id)
"window" window
"componentType" (-> comp-page-info .get_component_type str lower-case)
"windowHint" window-hint
@@ -859,7 +870,7 @@
(.get_eventlog_host comp-page-info)
(.get_eventlog_port comp-page-info)
secure?)
- "profilingAndDebuggingCapable" (not on-windows?)
+ "profilingAndDebuggingCapable" (not (Utils/isOnWindows))
"profileActionEnabled" (*STORM-CONF* WORKER-PROFILER-ENABLED)
"profilerActive" (if (*STORM-CONF* WORKER-PROFILER-ENABLED)
(get-active-profile-actions nimbus topology-id component)
@@ -960,7 +971,7 @@
(assert-authorized-user "getClusterInfo")
(json-response (all-topologies-summary) (:callback m)))
(GET "/api/v1/topology-workers/:id" [:as {:keys [cookies servlet-request]} id & m]
- (let [id (url-decode id)]
+ (let [id (URLDecoder/decode id)]
(json-response {"hostPortList" (worker-host-port id)
"logviewerPort" (*STORM-CONF* LOGVIEWER-PORT)} (:callback m))))
(GET "/api/v1/topology/:id" [:as {:keys [cookies servlet-request scheme]} id & m]
diff --git a/storm-core/src/clj/org/apache/storm/ui/helpers.clj b/storm-core/src/clj/org/apache/storm/ui/helpers.clj
index 7ded1540170..4da5804dd7a 100644
--- a/storm-core/src/clj/org/apache/storm/ui/helpers.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/helpers.clj
@@ -20,18 +20,20 @@
[string :only [blank? join]]
[walk :only [keywordize-keys]]])
(:use [org.apache.storm config log])
- (:use [org.apache.storm.util :only [clojurify-structure uuid defnk to-json url-encode not-nil?]])
+ (:use [org.apache.storm.util :only [clojurify-structure defnk not-nil?]])
(:use [clj-time coerce format])
(:import [org.apache.storm.generated ExecutorInfo ExecutorSummary])
(:import [org.apache.storm.logging.filters AccessLoggingFilter])
- (:import [java.util EnumSet])
+ (:import [java.util EnumSet]
+ [java.net URLEncoder])
(:import [org.eclipse.jetty.server Server]
[org.eclipse.jetty.server.nio SelectChannelConnector]
[org.eclipse.jetty.server.ssl SslSocketConnector]
[org.eclipse.jetty.servlet ServletHolder FilterMapping]
- [org.eclipse.jetty.util.ssl SslContextFactory]
+ [org.eclipse.jetty.util.ssl SslContextFactory]
[org.eclipse.jetty.server DispatcherType]
- [org.eclipse.jetty.servlets CrossOriginFilter])
+ [org.eclipse.jetty.servlets CrossOriginFilter]
+ (org.json.simple JSONValue))
(:require [ring.util servlet])
(:require [compojure.route :as route]
[compojure.handler :as handler])
@@ -108,7 +110,7 @@
(defn url-format [fmt & args]
(String/format fmt
- (to-array (map #(url-encode (str %)) args))))
+ (to-array (map #(URLEncoder/encode (str %)) args))))
(defn pretty-executor-info [^ExecutorInfo e]
(str "[" (.get_task_start e) "-" (.get_task_end e) "]"))
@@ -219,7 +221,7 @@
(str callback "(" response ");"))
(defnk json-response
- [data callback :serialize-fn to-json :status 200 :headers {}]
+ [data callback :serialize-fn #(JSONValue/toJSONString %) :status 200 :headers {}]
{:status status
:headers (merge {"Cache-Control" "no-cache, no-store"
"Access-Control-Allow-Origin" "*"
diff --git a/storm-core/src/clj/org/apache/storm/util.clj b/storm-core/src/clj/org/apache/storm/util.clj
index 23d39f672c0..f685d12aca8 100644
--- a/storm-core/src/clj/org/apache/storm/util.clj
+++ b/storm-core/src/clj/org/apache/storm/util.clj
@@ -20,8 +20,8 @@
(:import [java.io FileReader FileNotFoundException])
(:import [java.nio.file Paths])
(:import [org.apache.storm Config])
- (:import [org.apache.storm.utils Time Container ClojureTimerTask Utils
- MutableObject MutableInt])
+ (:import [org.apache.storm.utils Time ClojureTimerTask Utils
+ MutableObject])
(:import [org.apache.storm.security.auth NimbusPrincipal])
(:import [javax.security.auth Subject])
(:import [java.util UUID Random ArrayList List Collections])
@@ -35,7 +35,6 @@
(:import [java.lang.management ManagementFactory])
(:import [org.apache.commons.exec DefaultExecutor CommandLine])
(:import [org.apache.commons.io FileUtils])
- (:import [org.apache.storm.logging ThriftAccessLogger])
(:import [org.apache.commons.exec ExecuteException])
(:import [org.json.simple JSONValue])
(:import [org.yaml.snakeyaml Yaml]
@@ -48,25 +47,6 @@
(:require [ring.util.codec :as codec])
(:use [org.apache.storm log]))
-(defn wrap-in-runtime
- "Wraps an exception in a RuntimeException if needed"
- [^Exception e]
- (if (instance? RuntimeException e)
- e
- (RuntimeException. e)))
-
-(def on-windows?
- (= "Windows_NT" (System/getenv "OS")))
-
-(def file-path-separator
- (System/getProperty "file.separator"))
-
-(def class-path-separator
- (System/getProperty "path.separator"))
-
-(defn is-absolute-path? [path]
- (.isAbsolute (Paths/get path (into-array String []))))
-
(defmacro defalias
"Defines an alias for a var: a new var with the same root binding (if
any) and similar metadata. The metadata of the alias is its initial
@@ -130,64 +110,13 @@
(let [~de-map (apply hash-map options#)]
~@body))))
-(defn find-first
- "Returns the first item of coll for which (pred item) returns logical true.
- Consumes sequences up to the first match, will consume the entire sequence
- and return nil if no match is found."
- [pred coll]
- (first (filter pred coll)))
-
-(defn dissoc-in
- "Dissociates an entry from a nested associative structure returning a new
- nested structure. keys is a sequence of keys. Any empty maps that result
- will not be present in the new structure."
- [m [k & ks :as keys]]
- (if ks
- (if-let [nextmap (get m k)]
- (let [newmap (dissoc-in nextmap ks)]
- (if (seq newmap)
- (assoc m k newmap)
- (dissoc m k)))
- m)
- (dissoc m k)))
-
-(defn indexed
- "Returns a lazy sequence of [index, item] pairs, where items come
- from 's' and indexes count up from zero.
-
- (indexed '(a b c d)) => ([0 a] [1 b] [2 c] [3 d])"
- [s]
- (map vector (iterate inc 0) s))
-
-(defn positions
- "Returns a lazy sequence containing the positions at which pred
- is true for items in coll."
- [pred coll]
- (for [[idx elt] (indexed coll) :when (pred elt)] idx))
-
-(defn exception-cause?
- [klass ^Throwable t]
- (->> (iterate #(.getCause ^Throwable %) t)
- (take-while identity)
- (some (partial instance? klass))
- boolean))
-
(defmacro thrown-cause?
[klass & body]
`(try
~@body
false
(catch Throwable t#
- (exception-cause? ~klass t#))))
-
-(defmacro thrown-cause-with-msg?
- [klass re & body]
- `(try
- ~@body
- false
- (catch Throwable t#
- (and (re-matches ~re (.getMessage t#))
- (exception-cause? ~klass t#)))))
+ (Utils/exceptionCauseIsInstanceOf ~klass t#))))
(defmacro forcat
[[args aseq] & body]
@@ -203,7 +132,7 @@
[code guards] (split-with checker body)
error-local (gensym "t")
guards (forcat [[_ klass local & guard-body] guards]
- `((exception-cause? ~klass ~error-local)
+ `((Utils/exceptionCauseIsInstanceOf ~klass ~error-local)
(let [~local ~error-local]
~@guard-body
)))]
@@ -213,18 +142,6 @@
true (throw ~error-local)
)))))
-(defn local-hostname
- []
- (.getCanonicalHostName (InetAddress/getLocalHost)))
-
-(def memoized-local-hostname (memoize local-hostname))
-
-;; checks conf for STORM_LOCAL_HOSTNAME.
-;; when unconfigured, falls back to (memoized) guess by `local-hostname`.
-(defn hostname
- [conf]
- (conf Config/STORM_LOCAL_HOSTNAME (memoized-local-hostname)))
-
(letfn [(try-port [port]
(with-open [socket (java.net.ServerSocket. port)]
(.getLocalPort socket)))]
@@ -236,21 +153,6 @@
(catch java.io.IOException e
(available-port))))))
-(defn uuid []
- (str (UUID/randomUUID)))
-
-(defn current-time-secs
- []
- (Time/currentTimeSecs))
-
-(defn current-time-millis
- []
- (Time/currentTimeMillis))
-
-(defn secs-to-millis-long
- [secs]
- (long (* (long 1000) secs)))
-
(defn clojurify-structure
[s]
(prewalk (fn [x]
@@ -262,701 +164,48 @@
true x))
s))
-(defmacro with-file-lock
- [path & body]
- `(let [f# (File. ~path)
- _# (.createNewFile f#)
- rf# (RandomAccessFile. f# "rw")
- lock# (.. rf# (getChannel) (lock))]
- (try
- ~@body
- (finally
- (.release lock#)
- (.close rf#)))))
-
-(defn tokenize-path
- [^String path]
- (let [toks (.split path "/")]
- (vec (filter (complement empty?) toks))))
-
-(defn assoc-conj
- [m k v]
- (merge-with concat m {k [v]}))
-
-;; returns [ones in first set not in second, ones in second set not in first]
-(defn set-delta
- [old curr]
- (let [s1 (set old)
- s2 (set curr)]
- [(set/difference s1 s2) (set/difference s2 s1)]))
-
-(defn parent-path
- [path]
- (let [toks (tokenize-path path)]
- (str "/" (str/join "/" (butlast toks)))))
-
-(defn toks->path
- [toks]
- (str "/" (str/join "/" toks)))
-
-(defn normalize-path
- [^String path]
- (toks->path (tokenize-path path)))
-
+;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
(defn map-val
[afn amap]
(into {}
(for [[k v] amap]
[k (afn v)])))
+;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
(defn filter-val
[afn amap]
(into {} (filter (fn [[k v]] (afn v)) amap)))
+;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
(defn filter-key
[afn amap]
(into {} (filter (fn [[k v]] (afn k)) amap)))
+;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
(defn map-key
[afn amap]
(into {} (for [[k v] amap] [(afn k) v])))
-(defn separate
- [pred aseq]
- [(filter pred aseq) (filter (complement pred) aseq)])
-
-(defn full-path
- [parent name]
- (let [toks (tokenize-path parent)]
- (toks->path (conj toks name))))
-
+;TODO: Once all the other clojure functions (100+ locations) are translated to java, this function becomes moot.
(def not-nil? (complement nil?))
-(defn barr
- [& vals]
- (byte-array (map byte vals)))
-
-(defn exit-process!
- [val & msg]
- (log-error (RuntimeException. (str msg)) "Halting process: " msg)
- (.exit (Runtime/getRuntime) val))
-
-(defn sum
- [vals]
- (reduce + vals))
-
-(defn repeat-seq
- ([aseq]
- (apply concat (repeat aseq)))
- ([amt aseq]
- (apply concat (repeat amt aseq))))
-
-(defn div
- "Perform floating point division on the arguments."
- [f & rest]
- (apply / (double f) rest))
-
-(defn defaulted
- [val default]
- (if val val default))
-
-(defn mk-counter
- ([] (mk-counter 1))
- ([start-val]
- (let [val (atom (dec start-val))]
- (fn [] (swap! val inc)))))
-
-(defmacro for-times [times & body]
- `(for [i# (range ~times)]
- ~@body))
-
(defmacro dofor [& body]
`(doall (for ~@body)))
-(defn reverse-map
- "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}"
- [amap]
- (reduce (fn [m [k v]]
- (let [existing (get m v [])]
- (assoc m v (conj existing k))))
- {} amap))
-
-(defmacro print-vars [& vars]
- (let [prints (for [v vars] `(println ~(str v) ~v))]
- `(do ~@prints)))
-
-(defn process-pid
- "Gets the pid of this JVM. Hacky because Java doesn't provide a real way to do this."
- []
- (let [name (.getName (ManagementFactory/getRuntimeMXBean))
- split (.split name "@")]
- (when-not (= 2 (count split))
- (throw (RuntimeException. (str "Got unexpected process name: " name))))
- (first split)))
-
-(defn exec-command! [command]
- (let [[comm-str & args] (seq (.split command " "))
- command (CommandLine. comm-str)]
- (doseq [a args]
- (.addArgument command a))
- (.execute (DefaultExecutor.) command)))
-
-(defn extract-dir-from-jar [jarpath dir destdir]
- (try-cause
- (with-open [jarpath (ZipFile. jarpath)]
- (let [entries (enumeration-seq (.entries jarpath))]
- (doseq [file (filter (fn [entry](and (not (.isDirectory entry)) (.startsWith (.getName entry) dir))) entries)]
- (.mkdirs (.getParentFile (File. destdir (.getName file))))
- (with-open [out (FileOutputStream. (File. destdir (.getName file)))]
- (io/copy (.getInputStream jarpath file) out)))))
- (catch IOException e
- (log-message "Could not extract " dir " from " jarpath))))
-
-(defn sleep-secs [secs]
- (when (pos? secs)
- (Time/sleep (* (long secs) 1000))))
-
-(defn sleep-until-secs [target-secs]
- (Time/sleepUntil (* (long target-secs) 1000)))
-
-(def ^:const sig-kill 9)
-
-(def ^:const sig-term 15)
-
-(defn send-signal-to-process
- [pid signum]
- (try-cause
- (exec-command! (str (if on-windows?
- (if (== signum sig-kill) "taskkill /f /pid " "taskkill /pid ")
- (str "kill -" signum " "))
- pid))
- (catch ExecuteException e
- (log-message "Error when trying to kill " pid ". Process is probably already dead."))))
-
-(defn read-and-log-stream
- [prefix stream]
- (try
- (let [reader (BufferedReader. (InputStreamReader. stream))]
- (loop []
- (if-let [line (.readLine reader)]
- (do
- (log-warn (str prefix ":" line))
- (recur)))))
- (catch IOException e
- (log-warn "Error while trying to log stream" e))))
-
-(defn force-kill-process
- [pid]
- (send-signal-to-process pid sig-kill))
-
-(defn kill-process-with-sig-term
- [pid]
- (send-signal-to-process pid sig-term))
-
-(defn add-shutdown-hook-with-force-kill-in-1-sec
- "adds the user supplied function as a shutdown hook for cleanup.
- Also adds a function that sleeps for a second and then sends kill -9 to process to avoid any zombie process in case
- cleanup function hangs."
- [func]
- (.addShutdownHook (Runtime/getRuntime) (Thread. #(func)))
- (.addShutdownHook (Runtime/getRuntime) (Thread. #((sleep-secs 1)
- (.halt (Runtime/getRuntime) 20)))))
-
-(defprotocol SmartThread
- (start [this])
- (join [this])
- (interrupt [this])
- (sleeping? [this]))
-
-;; afn returns amount of time to sleep
-(defnk async-loop [afn
- :daemon false
- :kill-fn (fn [error] (exit-process! 1 "Async loop died!"))
- :priority Thread/NORM_PRIORITY
- :factory? false
- :start true
- :thread-name nil]
- (let [thread (Thread.
- (fn []
- (try-cause
- (let [afn (if factory? (afn) afn)]
- (loop []
- (let [sleep-time (afn)]
- (when-not (nil? sleep-time)
- (sleep-secs sleep-time)
- (recur))
- )))
- (catch InterruptedException e
- (log-message "Async loop interrupted!")
- )
- (catch Throwable t
- (log-error t "Async loop died!")
- (kill-fn t)))))]
- (.setDaemon thread daemon)
- (.setPriority thread priority)
- (when thread-name
- (.setName thread (str (.getName thread) "-" thread-name)))
- (when start
- (.start thread))
- ;; should return object that supports stop, interrupt, join, and waiting?
- (reify SmartThread
- (start
- [this]
- (.start thread))
- (join
- [this]
- (.join thread))
- (interrupt
- [this]
- (.interrupt thread))
- (sleeping?
- [this]
- (Time/isThreadWaiting thread)))))
-
-(defn shell-cmd
- [command]
- (->> command
- (map #(str \' (clojure.string/escape % {\' "'\"'\"'"}) \'))
- (clojure.string/join " ")))
-
-(defn script-file-path [dir]
- (str dir file-path-separator "storm-worker-script.sh"))
-
-(defn container-file-path [dir]
- (str dir file-path-separator "launch_container.sh"))
-
-(defnk write-script
- [dir command :environment {}]
- (let [script-src (str "#!/bin/bash\n" (clojure.string/join "" (map (fn [[k v]] (str (shell-cmd ["export" (str k "=" v)]) ";\n")) environment)) "\nexec " (shell-cmd command) ";")
- script-path (script-file-path dir)
- _ (spit script-path script-src)]
- script-path
- ))
-
-(defnk launch-process
- [command :environment {} :log-prefix nil :exit-code-callback nil :directory nil]
- (let [builder (ProcessBuilder. command)
- process-env (.environment builder)]
- (when directory (.directory builder directory))
- (.redirectErrorStream builder true)
- (doseq [[k v] environment]
- (.put process-env k v))
- (let [process (.start builder)]
- (if (or log-prefix exit-code-callback)
- (async-loop
- (fn []
- (if log-prefix
- (read-and-log-stream log-prefix (.getInputStream process)))
- (when exit-code-callback
- (try
- (.waitFor process)
- (catch InterruptedException e
- (log-message log-prefix " interrupted.")))
- (exit-code-callback (.exitValue process)))
- nil)))
- process)))
-
-(defn exists-file?
- [path]
- (.exists (File. path)))
-
-(defn rmr
- [path]
- (log-debug "Rmr path " path)
- (when (exists-file? path)
- (try
- (FileUtils/forceDelete (File. path))
- (catch FileNotFoundException e))))
-
-(defn rmpath
- "Removes file or directory at the path. Not recursive. Throws exception on failure"
- [path]
- (log-debug "Removing path " path)
- (when (exists-file? path)
- (let [deleted? (.delete (File. path))]
- (when-not deleted?
- (throw (RuntimeException. (str "Failed to delete " path)))))))
-
-(defn local-mkdirs
- [path]
- (log-debug "Making dirs at " path)
- (FileUtils/forceMkdir (File. path)))
-
-(defn touch
- [path]
- (log-debug "Touching file at " path)
- (let [success? (do (if on-windows? (.mkdirs (.getParentFile (File. path))))
- (.createNewFile (File. path)))]
- (when-not success?
- (throw (RuntimeException. (str "Failed to touch " path))))))
-
-(defn create-symlink!
- "Create symlink is to the target"
- ([path-dir target-dir file-name]
- (create-symlink! path-dir target-dir file-name file-name))
- ([path-dir target-dir from-file-name to-file-name]
- (let [path (str path-dir file-path-separator from-file-name)
- target (str target-dir file-path-separator to-file-name)
- empty-array (make-array String 0)
- attrs (make-array FileAttribute 0)
- abs-path (.toAbsolutePath (Paths/get path empty-array))
- abs-target (.toAbsolutePath (Paths/get target empty-array))]
- (log-debug "Creating symlink [" abs-path "] to [" abs-target "]")
- (if (not (.exists (.toFile abs-path)))
- (Files/createSymbolicLink abs-path abs-target attrs)))))
-
-(defn read-dir-contents
- [dir]
- (if (exists-file? dir)
- (let [content-files (.listFiles (File. dir))]
- (map #(.getName ^File %) content-files))
- []))
-
-(defn compact
- [aseq]
- (filter (complement nil?) aseq))
-
-(defn current-classpath
- []
- (System/getProperty "java.class.path"))
-
-(defn get-full-jars
- [dir]
- (map #(str dir file-path-separator %) (filter #(.endsWith % ".jar") (read-dir-contents dir))))
-
-(defn worker-classpath
- []
- (let [storm-dir (System/getProperty "storm.home")
- storm-lib-dir (str storm-dir file-path-separator "lib")
- storm-conf-dir (if-let [confdir (System/getenv "STORM_CONF_DIR")]
- confdir
- (str storm-dir file-path-separator "conf"))
- storm-extlib-dir (str storm-dir file-path-separator "extlib")
- extcp (System/getenv "STORM_EXT_CLASSPATH")]
- (if (nil? storm-dir)
- (current-classpath)
- (str/join class-path-separator
- (remove nil? (concat (get-full-jars storm-lib-dir) (get-full-jars storm-extlib-dir) [extcp] [storm-conf-dir]))))))
-
-(defn add-to-classpath
- [classpath paths]
- (if (empty? paths)
- classpath
- (str/join class-path-separator (cons classpath paths))))
-
-(defn ^ReentrantReadWriteLock mk-rw-lock
- []
- (ReentrantReadWriteLock.))
-
-(defmacro read-locked
- [rw-lock & body]
- (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})]
- `(let [rlock# (.readLock ~lock)]
- (try (.lock rlock#)
- ~@body
- (finally (.unlock rlock#))))))
-
-(defmacro write-locked
- [rw-lock & body]
- (let [lock (with-meta rw-lock {:tag `ReentrantReadWriteLock})]
- `(let [wlock# (.writeLock ~lock)]
- (try (.lock wlock#)
- ~@body
- (finally (.unlock wlock#))))))
-
-(defn time-delta
- [time-secs]
- (- (current-time-secs) time-secs))
-
-(defn time-delta-ms
- [time-ms]
- (- (System/currentTimeMillis) (long time-ms)))
-
-(defn parse-int
- [str]
- (Integer/valueOf str))
-
-(defn integer-divided
- [sum num-pieces]
- (clojurify-structure (Utils/integerDivided sum num-pieces)))
-
-(defn collectify
- [obj]
- (if (or (sequential? obj) (instance? Collection obj))
- obj
- [obj]))
-
-(defn to-json
- [obj]
- (JSONValue/toJSONString obj))
-
-(defn from-json
- [^String str]
- (if str
- (clojurify-structure
- (JSONValue/parse str))
- nil))
-
-(defmacro letlocals
- [& body]
- (let [[tobind lexpr] (split-at (dec (count body)) body)
- binded (vec (mapcat (fn [e]
- (if (and (list? e) (= 'bind (first e)))
- [(second e) (last e)]
- ['_ e]
- ))
- tobind))]
- `(let ~binded
- ~(first lexpr))))
-
-(defn remove-first
- [pred aseq]
- (let [[b e] (split-with (complement pred) aseq)]
- (when (empty? e)
- (throw (IllegalArgumentException. "Nothing to remove")))
- (concat b (rest e))))
-
-(defn assoc-non-nil
- [m k v]
- (if v (assoc m k v) m))
-
-(defn multi-set
- "Returns a map of elem to count"
- [aseq]
- (apply merge-with +
- (map #(hash-map % 1) aseq)))
-
-(defn set-var-root*
- [avar val]
- (alter-var-root avar (fn [avar] val)))
-
-(defmacro set-var-root
- [var-sym val]
- `(set-var-root* (var ~var-sym) ~val))
-
-(defmacro with-var-roots
- [bindings & body]
- (let [settings (partition 2 bindings)
- tmpvars (repeatedly (count settings) (partial gensym "old"))
- vars (map first settings)
- savevals (vec (mapcat (fn [t v] [t v]) tmpvars vars))
- setters (for [[v s] settings] `(set-var-root ~v ~s))
- restorers (map (fn [v s] `(set-var-root ~v ~s)) vars tmpvars)]
- `(let ~savevals
- ~@setters
- (try
- ~@body
- (finally
- ~@restorers)))))
-
-(defn map-diff
- "Returns mappings in m2 that aren't in m1"
- [m1 m2]
- (into {} (filter (fn [[k v]] (not= v (m1 k))) m2)))
-
-(defn select-keys-pred
- [pred amap]
- (into {} (filter (fn [[k v]] (pred k)) amap)))
-
-(defn rotating-random-range
- [choices]
- (let [rand (Random.)
- choices (ArrayList. choices)]
- (Collections/shuffle choices rand)
- [(MutableInt. -1) choices rand]))
-
-(defn acquire-random-range-id
- [[^MutableInt curr ^List state ^Random rand]]
- (when (>= (.increment curr) (.size state))
- (.set curr 0)
- (Collections/shuffle state rand))
- (.get state (.get curr)))
-
-; this can be rewritten to be tail recursive
-(defn interleave-all
- [& colls]
- (if (empty? colls)
- []
- (let [colls (filter (complement empty?) colls)
- my-elems (map first colls)
- rest-elems (apply interleave-all (map rest colls))]
- (concat my-elems rest-elems))))
-
-(defn any-intersection
- [& sets]
- (let [elem->count (multi-set (apply concat sets))]
- (-> (filter-val #(> % 1) elem->count)
- keys)))
-
-(defn between?
- "val >= lower and val <= upper"
- [val lower upper]
- (and (>= val lower)
- (<= val upper)))
-
-(defmacro benchmark
- [& body]
- `(let [l# (doall (range 1000000))]
- (time
- (doseq [i# l#]
- ~@body))))
-
-(defn rand-sampler
- [freq]
- (let [r (java.util.Random.)]
- (fn [] (= 0 (.nextInt r freq)))))
-
-(defn even-sampler
- [freq]
- (let [freq (int freq)
- start (int 0)
- r (java.util.Random.)
- curr (MutableInt. -1)
- target (MutableInt. (.nextInt r freq))]
- (with-meta
- (fn []
- (let [i (.increment curr)]
- (when (>= i freq)
- (.set curr start)
- (.set target (.nextInt r freq))))
- (= (.get curr) (.get target)))
- {:rate freq})))
-
-(defn sampler-rate
- [sampler]
- (:rate (meta sampler)))
-
-(defn class-selector
- [obj & args]
- (class obj))
-
-(defn uptime-computer []
- (let [start-time (current-time-secs)]
- (fn [] (time-delta start-time))))
-
-(defn stringify-error [error]
- (let [result (StringWriter.)
- printer (PrintWriter. result)]
- (.printStackTrace error printer)
- (.toString result)))
-
-(defn nil-to-zero
- [v]
- (or v 0))
-
-(defn bit-xor-vals
- [vals]
- (reduce bit-xor 0 vals))
-
-(defmacro with-error-reaction
- [afn & body]
- `(try ~@body
- (catch Throwable t# (~afn t#))))
-
-(defn container
- []
- (Container.))
-
-(defn container-set! [^Container container obj]
- (set! (. container object) obj)
- container)
-
-(defn container-get [^Container container]
- (. container object))
-
-(defn to-millis [secs]
- (* 1000 (long secs)))
-
-(defn throw-runtime [& strs]
- (throw (RuntimeException. (apply str strs))))
-
-(defn redirect-stdio-to-slf4j!
- []
- ;; set-var-root doesn't work with *out* and *err*, so digging much deeper here
- ;; Unfortunately, this code seems to work at the REPL but not when spawned as worker processes
- ;; it might have something to do with being a child process
- ;; (set! (. (.getThreadBinding RT/OUT) val)
- ;; (java.io.OutputStreamWriter.
- ;; (log-stream :info "STDIO")))
- ;; (set! (. (.getThreadBinding RT/ERR) val)
- ;; (PrintWriter.
- ;; (java.io.OutputStreamWriter.
- ;; (log-stream :error "STDIO"))
- ;; true))
- (log-capture! "STDIO"))
-
-(defn spy
- [prefix val]
- (log-message prefix ": " val)
- val)
-
-(defn zip-contains-dir?
- [zipfile target]
- (let [entries (->> zipfile (ZipFile.) .entries enumeration-seq (map (memfn getName)))]
- (boolean (some #(.startsWith % (str target "/")) entries))))
-
-(defn url-encode
- [s]
- (codec/url-encode s))
-
-(defn url-decode
- [s]
- (codec/url-decode s))
-
-(defn join-maps
- [& maps]
- (let [all-keys (apply set/union (for [m maps] (-> m keys set)))]
- (into {} (for [k all-keys]
- [k (for [m maps] (m k))]))))
-
-(defn partition-fixed
- [max-num-chunks aseq]
- (if (zero? max-num-chunks)
- []
- (let [chunks (->> (integer-divided (count aseq) max-num-chunks)
- (#(dissoc % 0))
- (sort-by (comp - first))
- (mapcat (fn [[size amt]] (repeat amt size)))
- )]
- (loop [result []
- [chunk & rest-chunks] chunks
- data aseq]
- (if (nil? chunk)
- result
- (let [[c rest-data] (split-at chunk data)]
- (recur (conj result c)
- rest-chunks
- rest-data)))))))
-
-
-(defn assoc-apply-self
- [curr key afn]
+;; The following two will go away when worker, task, executor go away.
+(defn assoc-apply-self [curr key afn]
(assoc curr key (afn curr)))
+; These seven following will go away later. To be replaced by idiomatic java.
(defmacro recursive-map
[& forms]
- (->> (partition 2 forms)
- (map (fn [[key form]] `(assoc-apply-self ~key (fn [~'<>] ~form))))
- (concat `(-> {}))))
-
-(defn current-stack-trace
- []
- (->> (Thread/currentThread)
- .getStackTrace
- (map str)
- (str/join "\n")))
-
-(defn get-iterator
- [^Iterable alist]
- (if alist (.iterator alist)))
-
-(defn iter-has-next?
- [^Iterator iter]
- (if iter (.hasNext iter) false))
-
-(defn iter-next
- [^Iterator iter]
- (.next iter))
+ (->> (partition 2 forms)
+ (map (fn [[key form]] `(assoc-apply-self ~key (fn [~'<>] ~form))))
+ (concat `(-> {}))))
(defmacro fast-list-iter
[pairs & body]
@@ -964,76 +213,46 @@
lists (map second pairs)
elems (map first pairs)
iters (map (fn [_] (gensym)) lists)
- bindings (->> (map (fn [i l] [i `(get-iterator ~l)]) iters lists)
+ bindings (->> (map (fn [i l] (let [lg (gensym)] [lg l i `(if ~lg (.iterator ~lg))])) iters lists)
(apply concat))
- tests (map (fn [i] `(iter-has-next? ~i)) iters)
- assignments (->> (map (fn [e i] [e `(iter-next ~i)]) elems iters)
+ tests (map (fn [i] `(and ~i (.hasNext ^Iterator ~i))) iters)
+ assignments (->> (map (fn [e i] [e `(.next ^Iterator ~i)]) elems iters)
(apply concat))]
`(let [~@bindings]
(while (and ~@tests)
(let [~@assignments]
~@body)))))
-(defn fast-list-map
- [afn alist]
- (let [ret (ArrayList.)]
- (fast-list-iter [e alist]
- (.add ret (afn e)))
- ret))
-
(defmacro fast-list-for
[[e alist] & body]
- `(fast-list-map (fn [~e] ~@body) ~alist))
-
-(defn map-iter
- [^Map amap]
- (if amap (-> amap .entrySet .iterator)))
-
-(defn convert-entry
- [^Map$Entry entry]
- [(.getKey entry) (.getValue entry)])
+ `(let [ret# (ArrayList.)]
+ (fast-list-iter [~e ~alist]
+ (.add ret# (do ~@body)))
+ ret#))
(defmacro fast-map-iter
[[bind amap] & body]
- `(let [iter# (map-iter ~amap)]
- (while (iter-has-next? iter#)
- (let [entry# (iter-next iter#)
- ~bind (convert-entry entry#)]
+ `(let [iter# (if ~amap (.. ^Map ~amap entrySet iterator))]
+ (while (and iter# (.hasNext ^Iterator iter#))
+ (let [entry# (.next ^Iterator iter#)
+ ~bind [(.getKey ^Map$Entry entry#) (.getValue ^Map$Entry entry#)]]
~@body))))
-(defn fast-first
- [^List alist]
- (.get alist 0))
-
-(defmacro get-with-default
- [amap key default-val]
- `(let [curr# (.get ~amap ~key)]
- (if curr#
- curr#
- (do
- (let [new# ~default-val]
- (.put ~amap ~key new#)
- new#)))))
-
(defn fast-group-by
[afn alist]
(let [ret (HashMap.)]
(fast-list-iter
[e alist]
(let [key (afn e)
- ^List curr (get-with-default ret key (ArrayList.))]
+ ^List curr (let [curr (.get ret key)]
+ (if curr
+ curr
+ (let [default (ArrayList.)]
+ (.put ret key default)
+ default)))]
(.add curr e)))
ret))
-(defn new-instance
- [klass]
- (let [klass (if (string? klass) (Class/forName klass) klass)]
- (.newInstance klass)))
-
-(defn get-configured-class
- [conf config-key]
- (if (.get conf config-key) (new-instance (.get conf config-key)) nil))
-
(defmacro -<>
([x] x)
([x form] (if (seq? form)
@@ -1044,75 +263,5 @@
(list form x)))
([x form & more] `(-<> (-<> ~x ~form) ~@more)))
-(defn logs-filename
- [storm-id port]
- (str storm-id file-path-separator port file-path-separator "worker.log"))
-
-(def worker-log-filename-pattern #"^worker.log(.*)")
-
-(defn event-logs-filename
- [storm-id port]
- (str storm-id file-path-separator port file-path-separator "events.log"))
-
-(defn clojure-from-yaml-file [yamlFile]
- (try
- (with-open [reader (java.io.FileReader. yamlFile)]
- (clojurify-structure (.load (Yaml. (SafeConstructor.)) reader)))
- (catch Exception ex
- (log-error ex))))
-
(defn hashmap-to-persistent [^HashMap m]
(zipmap (.keySet m) (.values m)))
-
-(defn retry-on-exception
- "Retries specific function on exception based on retries count"
- [retries task-description f & args]
- (let [res (try {:value (apply f args)}
- (catch Exception e
- (if (<= 0 retries)
- (throw e)
- {:exception e})))]
- (if (:exception res)
- (do
- (log-error (:exception res) (str "Failed to " task-description ". Will make [" retries "] more attempts."))
- (recur (dec retries) task-description f args))
- (do
- (log-debug (str "Successful " task-description "."))
- (:value res)))))
-
-(defn setup-default-uncaught-exception-handler
- "Set a default uncaught exception handler to handle exceptions not caught in other threads."
- []
- (Thread/setDefaultUncaughtExceptionHandler
- (proxy [Thread$UncaughtExceptionHandler] []
- (uncaughtException [thread thrown]
- (try
- (Utils/handleUncaughtException thrown)
- (catch Error err
- (do
- (log-error err "Received error in main thread.. terminating server...")
- (.exit (Runtime/getRuntime) -2))))))))
-
-(defn redact-value
- "Hides value for k in coll for printing coll safely"
- [coll k]
- (if (contains? coll k)
- (assoc coll k (apply str (repeat (count (coll k)) "#")))
- coll))
-
-(defn log-thrift-access
- [request-id remoteAddress principal operation]
- (doto
- (ThriftAccessLogger.)
- (.log (str "Request ID: " request-id " access from: " remoteAddress " principal: " principal " operation: " operation))))
-
-(def DISALLOWED-KEY-NAME-STRS #{"/" "." ":" "\\"})
-
-(defn validate-key-name!
- [name]
- (if (some #(.contains name %) DISALLOWED-KEY-NAME-STRS)
- (throw (RuntimeException.
- (str "Key name cannot contain any of the following: " (pr-str DISALLOWED-KEY-NAME-STRS))))
- (if (clojure.string/blank? name)
- (throw (RuntimeException.
- ("Key name cannot be blank"))))))
diff --git a/storm-core/src/clj/org/apache/storm/zookeeper.clj b/storm-core/src/clj/org/apache/storm/zookeeper.clj
index 413ffd6571d..246d5db0cd5 100644
--- a/storm-core/src/clj/org/apache/storm/zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/zookeeper.clj
@@ -72,4 +72,3 @@
;; ))))
(.start fk)
fk))
-
diff --git a/storm-core/src/jvm/org/apache/storm/logging/ThriftAccessLogger.java b/storm-core/src/jvm/org/apache/storm/logging/ThriftAccessLogger.java
index cf23d620e94..9befb528248 100644
--- a/storm-core/src/jvm/org/apache/storm/logging/ThriftAccessLogger.java
+++ b/storm-core/src/jvm/org/apache/storm/logging/ThriftAccessLogger.java
@@ -16,12 +16,17 @@
* limitations under the License.
*/
package org.apache.storm.logging;
+import java.net.InetAddress;
+import java.security.Principal;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class ThriftAccessLogger {
- private static final Logger LOG = LoggerFactory.getLogger(ThriftAccessLogger.class);
- public void log(String logMessage) {
- LOG.info(logMessage);
- }
+ private static final Logger LOG = LoggerFactory.getLogger(ThriftAccessLogger.class);
+
+ public static void logAccess(Integer requestId, InetAddress remoteAddress,
+ Principal principal, String operation) {
+ LOG.info("Request ID: {} access from: {} principal: {} operation: {}",
+ requestId, remoteAddress, principal, operation);
+ }
}
diff --git a/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java b/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java
index 678a74255f1..4007138448a 100644
--- a/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/serialization/SerializationFactory.java
@@ -132,6 +132,21 @@ public static class IdDictionary {
Map> streamNametoId = new HashMap<>();
Map> streamIdToName = new HashMap<>();
+ /**
+ * "{:a 1 :b 2} -> {1 :a 2 :b}"
+ *
+ * Note: Only one key wins if there are duplicate values.
+ * Which key wins is indeterminate:
+ * "{:a 1 :b 1} -> {1 :a} *or* {1 :b}"
+ */
+ private static Map simpleReverseMap(Map map) {
+ Map ret = new HashMap();
+ for (Map.Entry entry : map.entrySet()) {
+ ret.put(entry.getValue(), entry.getKey());
+ }
+ return ret;
+ }
+
public IdDictionary(StormTopology topology) {
List componentNames = new ArrayList<>(topology.get_spouts().keySet());
componentNames.addAll(topology.get_bolts().keySet());
@@ -141,7 +156,7 @@ public IdDictionary(StormTopology topology) {
ComponentCommon common = Utils.getComponentCommon(topology, name);
List streams = new ArrayList<>(common.get_streams().keySet());
streamNametoId.put(name, idify(streams));
- streamIdToName.put(name, Utils.reverseMap(streamNametoId.get(name)));
+ streamIdToName.put(name, simpleReverseMap(streamNametoId.get(name)));
}
}
diff --git a/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java b/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java
index 54523f92416..1ac0249ac8a 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/ConfigUtils.java
@@ -44,27 +44,19 @@ public class ConfigUtils {
// A singleton instance allows us to mock delegated static methods in our
// tests by subclassing.
- private static final ConfigUtils INSTANCE = new ConfigUtils();
- private static ConfigUtils _instance = INSTANCE;
+ private static ConfigUtils _instance = new ConfigUtils();;
/**
* Provide an instance of this class for delegates to use. To mock out
* delegated methods, provide an instance of a subclass that overrides the
* implementation of the delegated method.
- *
- * @param u a ConfigUtils instance
+ * @param u a Utils instance
+ * @return the previously set instance
*/
- public static void setInstance(ConfigUtils u) {
+ public static ConfigUtils setInstance(ConfigUtils u) {
+ ConfigUtils oldInstance = _instance;
_instance = u;
- }
-
- /**
- * Resets the singleton instance to the default. This is helpful to reset
- * the class to its original functionality when mocking is no longer
- * desired.
- */
- public static void resetInstance() {
- _instance = INSTANCE;
+ return oldInstance;
}
public static String getLogDir() {
diff --git a/storm-core/src/jvm/org/apache/storm/utils/Container.java b/storm-core/src/jvm/org/apache/storm/utils/Container.java
index c3947d1b54a..d875731fd38 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/Container.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/Container.java
@@ -20,5 +20,14 @@
import java.io.Serializable;
public class Container implements Serializable {
- public Object object;
+ private Object object;
+
+ public Object get () {
+ return object;
+ }
+
+ public Container set (Object obj) {
+ object = obj;
+ return this;
+ }
}
diff --git a/storm-core/src/jvm/org/apache/storm/utils/TestUtils.java b/storm-core/src/jvm/org/apache/storm/utils/IPredicate.java
similarity index 65%
rename from storm-core/src/jvm/org/apache/storm/utils/TestUtils.java
rename to storm-core/src/jvm/org/apache/storm/utils/IPredicate.java
index 8ff08a98092..7e8669a8e74 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/TestUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/IPredicate.java
@@ -15,20 +15,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.storm.utils;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.retry.ExponentialBackoffRetry;
-import java.util.Map;
-
-public class TestUtils extends Utils {
-
- public static void testSetupBuilder(CuratorFrameworkFactory.Builder
- builder, String zkStr, Map conf, ZookeeperAuthInfo auth)
- {
- setupBuilder(builder, zkStr, conf, auth);
- }
-
+/**
+ * This interface is implemented by classes, instances of which can be passed
+ * into certain Util functions which test some collection for elements matching
+ * the IPredicate. (IPredicate.test(...) == true)
+ */
+public interface IPredicate {
+ boolean test (T obj);
}
diff --git a/storm-core/src/jvm/org/apache/storm/utils/NimbusClient.java b/storm-core/src/jvm/org/apache/storm/utils/NimbusClient.java
index af9aebdb8d7..f5bad6e202e 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/NimbusClient.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/NimbusClient.java
@@ -32,7 +32,7 @@
import java.util.List;
import java.util.Map;
-public class NimbusClient extends ThriftClient {
+public class NimbusClient extends ThriftClient implements AutoCloseable {
private Nimbus.Client _client;
private static final Logger LOG = LoggerFactory.getLogger(NimbusClient.class);
diff --git a/storm-core/src/jvm/org/apache/storm/utils/Time.java b/storm-core/src/jvm/org/apache/storm/utils/Time.java
index 17922525513..fd01fb88ef9 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/Time.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/Time.java
@@ -86,10 +86,16 @@ public static void sleepUntil(long targetTimeMs) throws InterruptedException {
Thread.sleep(sleepTime);
}
}
-
+
public static void sleep(long ms) throws InterruptedException {
sleepUntil(currentTimeMillis()+ms);
}
+
+ public static void sleepSecs (long secs) throws InterruptedException {
+ if (secs > 0) {
+ sleep(secs * 1000);
+ }
+ }
public static long currentTimeMillis() {
if(simulating.get()) {
@@ -98,10 +104,26 @@ public static long currentTimeMillis() {
return System.currentTimeMillis();
}
}
-
+
+ public static long secsToMillis (int secs) {
+ return 1000*(long) secs;
+ }
+
+ public static long secsToMillisLong(double secs) {
+ return (long) (1000 * secs);
+ }
+
public static int currentTimeSecs() {
return (int) (currentTimeMillis() / 1000);
}
+
+ public static int deltaSecs(int timeInSeconds) {
+ return Time.currentTimeSecs() - timeInSeconds;
+ }
+
+ public static long deltaMs(long timeInMilliseconds) {
+ return Time.currentTimeMillis() - timeInMilliseconds;
+ }
public static void advanceTime(long ms) {
if(!simulating.get()) throw new IllegalStateException("Cannot simulate time unless in simulation mode");
diff --git a/storm-core/src/jvm/org/apache/storm/utils/Utils.java b/storm-core/src/jvm/org/apache/storm/utils/Utils.java
index 380f4dd340c..4ec17921d6e 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/Utils.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/Utils.java
@@ -17,7 +17,11 @@
*/
package org.apache.storm.utils;
+import org.apache.commons.exec.CommandLine;
+import org.apache.commons.exec.DefaultExecutor;
+import org.apache.commons.exec.ExecuteException;
import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
import org.apache.storm.Config;
import org.apache.storm.blobstore.BlobStore;
import org.apache.storm.blobstore.BlobStoreAclHandler;
@@ -30,7 +34,6 @@
import org.apache.storm.nimbus.NimbusInfo;
import org.apache.storm.serialization.DefaultSerializationDelegate;
import org.apache.storm.serialization.SerializationDelegate;
-import clojure.lang.IFn;
import clojure.lang.RT;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
@@ -57,12 +60,38 @@
import org.yaml.snakeyaml.Yaml;
import org.yaml.snakeyaml.constructor.SafeConstructor;
-import java.io.*;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.RandomAccessFile;
+import java.io.Serializable;
+import java.lang.management.ManagementFactory;
+import java.net.InetAddress;
import java.net.URL;
import java.net.URLDecoder;
+import java.net.UnknownHostException;
import java.nio.ByteBuffer;
import java.nio.file.FileSystems;
import java.nio.file.Files;
+import java.nio.file.Path;
import java.nio.file.Paths;
import java.nio.file.attribute.PosixFilePermission;
import java.util.ArrayList;
@@ -73,11 +102,14 @@
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
+import java.util.LinkedList;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeMap;
import java.util.UUID;
+import java.util.concurrent.Callable;
import java.util.jar.JarEntry;
import java.util.jar.JarFile;
import java.util.regex.Matcher;
@@ -88,6 +120,23 @@
import java.util.zip.ZipFile;
public class Utils {
+ // A singleton instance allows us to mock delegated static methods in our
+ // tests by subclassing.
+ private static Utils _instance = new Utils();
+
+ /**
+ * Provide an instance of this class for delegates to use. To mock out
+ * delegated methods, provide an instance of a subclass that overrides the
+ * implementation of the delegated method.
+ * @param u a Utils instance
+ * @return the previously set instance
+ */
+ public static Utils setInstance(Utils u) {
+ Utils oldInstance = _instance;
+ _instance = u;
+ return oldInstance;
+ }
+
private static final Logger LOG = LoggerFactory.getLogger(Utils.class);
public static final String DEFAULT_STREAM_ID = "default";
public static final String DEFAULT_BLOB_VERSION_SUFFIX = ".version";
@@ -99,15 +148,34 @@ public class Utils {
private static SerializationDelegate serializationDelegate;
private static ClassLoader cl = ClassLoader.getSystemClassLoader();
+ public static final boolean IS_ON_WINDOWS = "Windows_NT".equals(System.getenv("OS"));
+ public static final String FILE_PATH_SEPARATOR = System.getProperty("file.separator");
+ public static final String CLASS_PATH_SEPARATOR = System.getProperty("path.separator");
+
+ public static final int SIGKILL = 9;
+ public static final int SIGTERM = 15;
+
static {
Map conf = readStormConfig();
serializationDelegate = getSerializationDelegate(conf);
}
- public static Object newInstance(String klass) {
+ public static T newInstance(String klass) {
try {
- Class c = Class.forName(klass);
- return c.newInstance();
+ return newInstance((Class)Class.forName(klass));
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static T newInstance(Class klass) {
+ return _instance.newInstanceImpl(klass);
+ }
+
+ // Non-static impl methods exist for mocking purposes.
+ public T newInstanceImpl(Class klass) {
+ try {
+ return klass.newInstance();
} catch (Exception e) {
throw new RuntimeException(e);
}
@@ -441,7 +509,11 @@ public static BlobStore getNimbusBlobStore(Map conf, String baseDir, NimbusInfo
HashMap nconf = new HashMap(conf);
// only enable cleanup of blobstore on nimbus
nconf.put(Config.BLOBSTORE_CLEANUP_ENABLE, Boolean.TRUE);
- store.prepare(nconf, baseDir, nimbusInfo);
+
+ if(store != null) {
+ // store can be null during testing when mocking utils.
+ store.prepare(nconf, baseDir, nimbusInfo);
+ }
return store;
}
@@ -514,6 +586,10 @@ private static boolean downloadResourcesAsSupervisorAttempt(ClientBlobStore cb,
return isSuccess;
}
+ public static boolean checkFileExists(String path) {
+ return Files.exists(new File(path).toPath());
+ }
+
public static boolean checkFileExists(String dir, String file) {
return Files.exists(new File(dir, file).toPath());
}
@@ -580,27 +656,19 @@ public static void restrictPermissions(String baseDir) {
}
- public static synchronized IFn loadClojureFn(String namespace, String name) {
+ public static synchronized clojure.lang.IFn loadClojureFn(String namespace, String name) {
try {
clojure.lang.Compiler.eval(RT.readString("(require '" + namespace + ")"));
} catch (Exception e) {
//if playing from the repl and defining functions, file won't exist
}
- return (IFn) RT.var(namespace, name).deref();
+ return (clojure.lang.IFn) RT.var(namespace, name).deref();
}
public static boolean isSystemId(String id) {
return id.startsWith("__");
}
- public static Map reverseMap(Map map) {
- Map ret = new HashMap();
- for (Map.Entry entry : map.entrySet()) {
- ret.put(entry.getValue(), entry.getKey());
- }
- return ret;
- }
-
public static ComponentCommon getComponentCommon(StormTopology topology, String id) {
if (topology.get_spouts().containsKey(id)) {
return topology.get_spouts().get(id).get_common();
@@ -828,7 +896,7 @@ public static void unTar(File inFile, File untarDir) throws IOException {
}
boolean gzipped = inFile.toString().endsWith("gz");
- if (onWindows()) {
+ if (isOnWindows()) {
// Tar is not native to Windows. Use simple Java based implementation for
// tests and simple tar archives
unTarUsingJava(inFile, untarDir, gzipped);
@@ -870,7 +938,6 @@ private static void unTarUsingTar(File inFile, File untarDir,
private static void unTarUsingJava(File inFile, File untarDir,
boolean gzipped) throws IOException {
InputStream inputStream = null;
- TarArchiveInputStream tis = null;
try {
if (gzipped) {
inputStream = new BufferedInputStream(new GZIPInputStream(
@@ -878,31 +945,15 @@ private static void unTarUsingJava(File inFile, File untarDir,
} else {
inputStream = new BufferedInputStream(new FileInputStream(inFile));
}
- tis = new TarArchiveInputStream(inputStream);
- for (TarArchiveEntry entry = tis.getNextTarEntry(); entry != null; ) {
- unpackEntries(tis, entry, untarDir);
- entry = tis.getNextTarEntry();
+ try (TarArchiveInputStream tis = new TarArchiveInputStream(inputStream)) {
+ for (TarArchiveEntry entry = tis.getNextTarEntry(); entry != null; ) {
+ unpackEntries(tis, entry, untarDir);
+ entry = tis.getNextTarEntry();
+ }
}
} finally {
- cleanup(tis, inputStream);
- }
- }
-
- /**
- * Close the Closeable objects and ignore any {@link IOException} or
- * null pointers. Must only be used for cleanup in exception handlers.
- *
- * @param closeables the objects to close
- */
- private static void cleanup(java.io.Closeable... closeables) {
- for (java.io.Closeable c : closeables) {
- if (c != null) {
- try {
- c.close();
- } catch (IOException e) {
- LOG.debug("Exception in closing " + c, e);
-
- }
+ if(inputStream != null) {
+ inputStream.close();
}
}
}
@@ -924,7 +975,7 @@ private static void unpackEntries(TarArchiveInputStream tis,
if (!outputFile.getParentFile().exists()) {
if (!outputFile.getParentFile().mkdirs()) {
throw new IOException("Mkdirs failed to create tar internal dir "
- + outputDir);
+ + outputDir);
}
}
int count;
@@ -939,13 +990,17 @@ private static void unpackEntries(TarArchiveInputStream tis,
outputStream.close();
}
- public static boolean onWindows() {
+ public static boolean isOnWindows() {
if (System.getenv("OS") != null) {
return System.getenv("OS").equals("Windows_NT");
}
return false;
}
+ public static boolean isAbsolutePath(String path) {
+ return Paths.get(path).isAbsolute();
+ }
+
public static void unpack(File localrsrc, File dst) throws IOException {
String lowerDst = localrsrc.getName().toLowerCase();
if (lowerDst.endsWith(".jar")) {
@@ -1034,6 +1089,12 @@ public String getBackupConnectionString() throws Exception {
}
}
+ public static void testSetupBuilder(CuratorFrameworkFactory.Builder
+ builder, String zkStr, Map conf, ZookeeperAuthInfo auth)
+ {
+ setupBuilder(builder, zkStr, conf, auth);
+ }
+
public static CuratorFramework newCurator(Map conf, List servers, Object port, ZookeeperAuthInfo auth) {
return newCurator(conf, servers, port, "", auth);
}
@@ -1076,10 +1137,16 @@ public static void readAndLogStream(String prefix, InputStream in) {
LOG.info("{}:{}", prefix, line);
}
} catch (IOException e) {
- LOG.warn("Error whiel trying to log stream", e);
+ LOG.warn("Error while trying to log stream", e);
}
}
+ /**
+ * Checks if a throwable is an instance of a particular class
+ * @param klass The class you're expecting
+ * @param throwable The throwable you expect to be an instance of klass
+ * @return true if throwable is instance of klass, false otherwise.
+ */
public static boolean exceptionCauseIsInstanceOf(Class klass, Throwable throwable) {
Throwable t = throwable;
while (t != null) {
@@ -1115,6 +1182,7 @@ public static boolean isZkAuthenticationConfiguredTopology(Map conf) {
&& !((String)conf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_SCHEME)).isEmpty());
}
+
public static List getWorkerACL(Map conf) {
//This is a work around to an issue with ZK where a sasl super user is not super unless there is an open SASL ACL so we are trying to give the correct perms
if (!isZkAuthenticationConfiguredTopology(conf)) {
@@ -1122,11 +1190,11 @@ public static List getWorkerACL(Map conf) {
}
String stormZKUser = (String)conf.get(Config.STORM_ZOOKEEPER_SUPERACL);
if (stormZKUser == null) {
- throw new IllegalArgumentException("Authentication is enabled but "+Config.STORM_ZOOKEEPER_SUPERACL+" is not set");
+ throw new IllegalArgumentException("Authentication is enabled but " + Config.STORM_ZOOKEEPER_SUPERACL + " is not set");
}
- String[] split = stormZKUser.split(":",2);
+ String[] split = stormZKUser.split(":", 2);
if (split.length != 2) {
- throw new IllegalArgumentException(Config.STORM_ZOOKEEPER_SUPERACL+" does not appear to be in the form scheme:acl, i.e. sasl:storm-user");
+ throw new IllegalArgumentException(Config.STORM_ZOOKEEPER_SUPERACL + " does not appear to be in the form scheme:acl, i.e. sasl:storm-user");
}
ArrayList ret = new ArrayList(ZooDefs.Ids.CREATOR_ALL_ACL);
ret.add(new ACL(ZooDefs.Perms.ALL, new Id(split[0], split[1])));
@@ -1165,6 +1233,10 @@ public static long getDU(File dir) {
}
}
+ /**
+ * Gets some information, including stack trace, for a running thread.
+ * @return A human-readable string of the dump.
+ */
public static String threadDump() {
final StringBuilder dump = new StringBuilder();
final java.lang.management.ThreadMXBean threadMXBean = java.lang.management.ManagementFactory.getThreadMXBean();
@@ -1186,20 +1258,19 @@ public static String threadDump() {
return dump.toString();
}
- // Assumes caller is synchronizing
+ /**
+ * Creates an instance of the pluggable SerializationDelegate or falls back to
+ * DefaultSerializationDelegate if something goes wrong.
+ * @param stormConf The config from which to pull the name of the pluggable class.
+ * @return an instance of the class specified by storm.meta.serialization.delegate
+ */
private static SerializationDelegate getSerializationDelegate(Map stormConf) {
String delegateClassName = (String)stormConf.get(Config.STORM_META_SERIALIZATION_DELEGATE);
SerializationDelegate delegate;
try {
Class delegateClass = Class.forName(delegateClassName);
delegate = (SerializationDelegate) delegateClass.newInstance();
- } catch (ClassNotFoundException e) {
- LOG.error("Failed to construct serialization delegate, falling back to default", e);
- delegate = new DefaultSerializationDelegate();
- } catch (InstantiationException e) {
- LOG.error("Failed to construct serialization delegate, falling back to default", e);
- delegate = new DefaultSerializationDelegate();
- } catch (IllegalAccessException e) {
+ } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
LOG.error("Failed to construct serialization delegate, falling back to default", e);
delegate = new DefaultSerializationDelegate();
}
@@ -1245,7 +1316,7 @@ public static void unZip(File inFile, File unzipDir) throws IOException {
if (!file.getParentFile().mkdirs()) {
if (!file.getParentFile().isDirectory()) {
throw new IOException("Mkdirs failed to create " +
- file.getParentFile().toString());
+ file.getParentFile().toString());
}
}
OutputStream out = new FileOutputStream(file);
@@ -1370,9 +1441,811 @@ public static int toPositive(int number) {
public static RuntimeException wrapInRuntime(Exception e){
if (e instanceof RuntimeException){
return (RuntimeException)e;
- }else {
+ } else {
return new RuntimeException(e);
}
}
-}
+ /**
+ * Determines if a zip archive contains a particular directory.
+ *
+ * @param zipfile path to the zipped file
+ * @param target directory being looked for in the zip.
+ * @return boolean whether or not the directory exists in the zip.
+ */
+ public static boolean zipDoesContainDir(String zipfile, String target) throws IOException {
+ List entries = (List)Collections.list(new ZipFile(zipfile).entries());
+
+ String targetDir = target + "/";
+ for(ZipEntry entry : entries) {
+ String name = entry.getName();
+ if(name.startsWith(targetDir)) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ /**
+ * Joins any number of maps together into a single map, combining their values into
+ * a list, maintaining values in the order the maps were passed in. Nulls are inserted
+ * for given keys when the map does not contain that key.
+ *
+ * i.e. joinMaps({'a' => 1, 'b' => 2}, {'b' => 3}, {'a' => 4, 'c' => 5}) ->
+ * {'a' => [1, null, 4], 'b' => [2, 3, null], 'c' => [null, null, 5]}
+ *
+ * @param maps variable number of maps to join - order affects order of values in output.
+ * @return combined map
+ */
+ public static Map> joinMaps(Map... maps) {
+ Map> ret = new HashMap<>();
+
+ Set keys = new HashSet<>();
+
+ for(Map map : maps) {
+ keys.addAll(map.keySet());
+ }
+
+ for(Map m : maps) {
+ for(K key : keys) {
+ V value = m.get(key);
+
+ if(!ret.containsKey(key)) {
+ ret.put(key, new ArrayList());
+ }
+
+ List targetList = ret.get(key);
+ targetList.add(value);
+ }
+ }
+ return ret;
+ }
+
+ /**
+ * Fills up chunks out of a collection (given a maximum amount of chunks)
+ *
+ * i.e. partitionFixed(5, [1,2,3]) -> [[1,2,3]]
+ * partitionFixed(5, [1..9]) -> [[1,2], [3,4], [5,6], [7,8], [9]]
+ * partitionFixed(3, [1..10]) -> [[1,2,3,4], [5,6,7], [8,9,10]]
+ * @param maxNumChunks the maximum number of chunks to return
+ * @param coll the collection to be chunked up
+ * @return a list of the chunks, which are themselves lists.
+ */
+ public static List> partitionFixed(int maxNumChunks, Collection coll) {
+ List> ret = new ArrayList<>();
+
+ if(maxNumChunks == 0 || coll == null) {
+ return ret;
+ }
+
+ Map parts = integerDivided(coll.size(), maxNumChunks);
+
+ // Keys sorted in descending order
+ List sortedKeys = new ArrayList(parts.keySet());
+ Collections.sort(sortedKeys, Collections.reverseOrder());
+
+
+ Iterator it = coll.iterator();
+ for(Integer chunkSize : sortedKeys) {
+ if(!it.hasNext()) { break; }
+ Integer times = parts.get(chunkSize);
+ for(int i = 0; i < times; i++) {
+ if(!it.hasNext()) { break; }
+ List chunkList = new ArrayList<>();
+ for(int j = 0; j < chunkSize; j++) {
+ if(!it.hasNext()) { break; }
+ chunkList.add(it.next());
+ }
+ ret.add(chunkList);
+ }
+ }
+
+ return ret;
+ }
+
+ /**
+ * Return a new instance of a pluggable specified in the conf.
+ * @param conf The conf to read from.
+ * @param configKey The key pointing to the pluggable class
+ * @return an instance of the class or null if it is not specified.
+ */
+ public static Object getConfiguredClass(Map conf, Object configKey) {
+ if (conf.containsKey(configKey)) {
+ return newInstance((String)conf.get(configKey));
+ }
+ return null;
+ }
+
+ public static String logsFilename(String stormId, int port) {
+ return stormId + FILE_PATH_SEPARATOR + port + FILE_PATH_SEPARATOR + "worker.log";
+ }
+
+ public static String eventLogsFilename(String stormId, int port) {
+ return stormId + FILE_PATH_SEPARATOR + port + FILE_PATH_SEPARATOR + "events.log";
+ }
+
+ public static Object readYamlFile(String yamlFile) {
+ try (FileReader reader = new FileReader(yamlFile)) {
+ return new Yaml(new SafeConstructor()).load(reader);
+ } catch(Exception ex) {
+ LOG.error("Failed to read yaml file.", ex);
+ }
+ return null;
+ }
+
+ public static void setupDefaultUncaughtExceptionHandler() {
+ Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+ public void uncaughtException(Thread thread, Throwable thrown) {
+ try {
+ handleUncaughtException(thrown);
+ } catch (Error err) {
+ LOG.error("Received error in main thread.. terminating server...", err);
+ Runtime.getRuntime().exit(-2);
+ }
+ }
+ });
+ }
+
+ /**
+ * Creates a new map with a string value in the map replaced with an
+ * equivalently-lengthed string of '#'.
+ * @param m The map that a value will be redacted from
+ * @param key The key pointing to the value to be redacted
+ * @return a new map with the value redacted. The original map will not be modified.
+ */
+ public static Map