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 redactValue(Map m, Object key) { + if(m.containsKey(key)) { + HashMap newMap = new HashMap<>(m); + String value = newMap.get(key); + String redacted = new String(new char[value.length()]).replace("\0", "#"); + newMap.put(key, redacted); + return newMap; + } + return m; + } + + /** + * Make sure a given key name is valid for the storm config. + * Throw RuntimeException if the key isn't valid. + * @param name The name of the config key to check. + */ + private static final Set disallowedKeys = new HashSet<>(Arrays.asList(new String[] {"/", ".", ":", "\\"})); + public static void validateKeyName(String name) { + + for(String key : disallowedKeys) { + if( name.contains(key) ) { + throw new RuntimeException("Key name cannot contain any of the following: " + disallowedKeys.toString()); + } + } + if(name.trim().isEmpty()) { + throw new RuntimeException("Key name cannot be blank"); + } + } + + /** + * Find the first item of coll for which pred.test(...) returns true. + * @param pred The IPredicate to test for + * @param coll The Collection of items to search through. + * @return The first matching value in coll, or null if nothing matches. + */ + public static T findOne (IPredicate pred, Collection coll) { + if(coll == null) { + return null; + } + for(T elem : coll) { + if (pred.test(elem)) { + return elem; + } + } + return null; + } + + public static T findOne (IPredicate pred, Map map) { + if(map == null) { + return null; + } + return findOne(pred, (Set)map.entrySet()); + } + + public static String localHostname () throws UnknownHostException { + return _instance.localHostnameImpl(); + } + + // Non-static impl methods exist for mocking purposes. + protected String localHostnameImpl () throws UnknownHostException { + return InetAddress.getLocalHost().getCanonicalHostName(); + } + + private static String memoizedLocalHostnameString = null; + + public static String memoizedLocalHostname () throws UnknownHostException { + if (memoizedLocalHostnameString == null) { + memoizedLocalHostnameString = localHostname(); + } + return memoizedLocalHostnameString; + } + + /** + * Gets the storm.local.hostname value, or tries to figure out the local hostname + * if it is not set in the config. + * @param conf The storm config to read from + * @return a string representation of the hostname. + */ + public static String hostname (Map conf) throws UnknownHostException { + if (conf == null) { + return memoizedLocalHostname(); + } + Object hostnameString = conf.get(Config.STORM_LOCAL_HOSTNAME); + if (hostnameString == null || hostnameString.equals("")) { + return memoizedLocalHostname(); + } + return (String)hostnameString; + } + + public static String uuid() { + return UUID.randomUUID().toString(); + } + + public static void exitProcess (int val, Object... msg) { + StringBuilder errorMessage = new StringBuilder(); + errorMessage.append("Halting process: "); + for (Object oneMessage: msg) { + errorMessage.append(oneMessage); + } + String combinedErrorMessage = errorMessage.toString(); + LOG.error(combinedErrorMessage, new RuntimeException(combinedErrorMessage)); + Runtime.getRuntime().exit(val); + } + + /** + * "{:a 1 :b 1 :c 2} -> {1 [:a :b] 2 :c}" + * + * Example usage in java: + * Map tasks; + * Map> componentTasks = Utils.reverse_map(tasks); + * + * The order of he resulting list values depends on the ordering properties + * of the Map passed in. The caller is responsible for passing an ordered + * map if they expect the result to be consistently ordered as well. + * + * @param map to reverse + * @return a reversed map + */ + public static HashMap> reverseMap(Map map) { + HashMap> rtn = new HashMap>(); + if (map == null) { + return rtn; + } + for (Entry entry : map.entrySet()) { + K key = entry.getKey(); + V val = entry.getValue(); + List list = rtn.get(val); + if (list == null) { + list = new ArrayList(); + rtn.put(entry.getValue(), list); + } + list.add(key); + } + return rtn; + } + + /** + * "[[:a 1] [:b 1] [:c 2]} -> {1 [:a :b] 2 :c}" + * Reverses an assoc-list style Map like reverseMap(Map...) + * + * @param listSeq to reverse + * @return a reversed map + */ + public static HashMap reverseMap(List listSeq) { + HashMap> rtn = new HashMap(); + if (listSeq == null) { + return rtn; + } + for (Object entry : listSeq) { + List listEntry = (List) entry; + Object key = listEntry.get(0); + Object val = listEntry.get(1); + List list = rtn.get(val); + if (list == null) { + list = new ArrayList(); + rtn.put(val, list); + } + list.add(key); + } + return rtn; + } + + + /** + * @return the pid of this JVM, because Java doesn't provide a real way to do this. + */ + public static String processPid() { + String name = ManagementFactory.getRuntimeMXBean().getName(); + String[] split = name.split("@"); + if (split.length != 2) { + throw new RuntimeException("Got unexpected process name: " + name); + } + return split[0]; + } + + public static int execCommand(String... command) throws ExecuteException, IOException { + CommandLine cmd = new CommandLine(command[0]); + for (int i = 1; i < command.length; i++) { + cmd.addArgument(command[i]); + } + + DefaultExecutor exec = new DefaultExecutor(); + return exec.execute(cmd); + } + + /** + * Extract dir from the jar to destdir + * + * @param jarpath Path to the jar file + * @param dir Directory in the jar to pull out + * @param destdir Path to the directory where the extracted directory will be put + * + */ + public static void extractDirFromJar(String jarpath, String dir, String destdir) { + try (JarFile jarFile = new JarFile(jarpath)) { + Enumeration jarEnums = jarFile.entries(); + while (jarEnums.hasMoreElements()) { + JarEntry entry = jarEnums.nextElement(); + if (!entry.isDirectory() && entry.getName().startsWith(dir)) { + File aFile = new File(destdir, entry.getName()); + aFile.getParentFile().mkdirs(); + try (FileOutputStream out = new FileOutputStream(aFile); + InputStream in = jarFile.getInputStream(entry)) { + IOUtils.copy(in, out); + } + } + } + } catch (IOException e) { + LOG.info("Could not extract {} from {}", dir, jarpath); + } + } + + public static void sendSignalToProcess(long lpid, int signum) throws IOException { + String pid = Long.toString(lpid); + try { + if (isOnWindows()) { + if (signum == SIGKILL) { + execCommand("taskkill", "/f", "/pid", pid); + } else { + execCommand("taskkill", "/pid", pid); + } + } else { + execCommand("kill", "-" + signum, pid); + } + } catch (ExecuteException e) { + LOG.info("Error when trying to kill {}. Process is probably already dead.", pid); + } catch (IOException e) { + LOG.info("IOException Error when trying to kill {}.", pid); + throw e; + } + } + + public static void forceKillProcess (String pid) throws IOException { + sendSignalToProcess(Long.parseLong(pid), SIGKILL); + } + + public static void killProcessWithSigTerm (String pid) throws IOException { + sendSignalToProcess(Long.parseLong(pid), SIGTERM); + } + + /** + * Adds the user supplied function as a shutdown hook for cleanup. + * Also adds a function that sleeps for a second and then halts the + * runtime to avoid any zombie process in case cleanup function hangs. + */ + public static void addShutdownHookWithForceKillIn1Sec (Runnable func) { + Runnable sleepKill = new Runnable() { + @Override + public void run() { + try { + Time.sleepSecs(1); + Runtime.getRuntime().halt(20); + } catch (Exception e) { + LOG.warn("Exception in the ShutDownHook", e); + } + } + }; + Runtime.getRuntime().addShutdownHook(new Thread(func)); + Runtime.getRuntime().addShutdownHook(new Thread(sleepKill)); + } + + /** + * Returns the combined string, escaped for posix shell. + * @param command the list of strings to be combined + * @return the resulting command string + */ + public static String shellCmd (List command) { + List changedCommands = new ArrayList<>(command.size()); + for (String str: command) { + if (str == null) { + continue; + } + changedCommands.add("'" + str.replaceAll("'", "'\"'\"'") + "'"); + } + return StringUtils.join(changedCommands, " "); + } + + public static String scriptFilePath (String dir) { + return dir + FILE_PATH_SEPARATOR + "storm-worker-script.sh"; + } + + public static String containerFilePath (String dir) { + return dir + FILE_PATH_SEPARATOR + "launch_container.sh"; + } + + public static Object nullToZero (Object v) { + return (v != null ? v : 0); + } + + /** + * Deletes a file or directory and its contents if it exists. Does not + * complain if the input is null or does not exist. + * @param path the path to the file or directory + */ + public static void forceDelete(String path) throws IOException { + _instance.forceDeleteImpl(path); + } + + // Non-static impl methods exist for mocking purposes. + protected void forceDeleteImpl(String path) throws IOException { + LOG.debug("Deleting path {}", path); + if (checkFileExists(path)) { + try { + FileUtils.forceDelete(new File(path)); + } catch (FileNotFoundException ignored) {} + } + } + + /** + * Creates a symbolic link to the target + * @param dir the parent directory of the link + * @param targetDir the parent directory of the link's target + * @param targetFilename the file name of the links target + * @param filename the file name of the link + * @throws IOException + */ + public static void createSymlink(String dir, String targetDir, + String targetFilename, String filename) throws IOException { + Path path = Paths.get(dir, filename).toAbsolutePath(); + Path target = Paths.get(targetDir, targetFilename).toAbsolutePath(); + LOG.debug("Creating symlink [{}] to [{}]", path, target); + if (!path.toFile().exists()) { + Files.createSymbolicLink(path, target); + } + } + + /** + * Convenience method for the case when the link's file name should be the + * same as the file name of the target + */ + public static void createSymlink(String dir, String targetDir, + String targetFilename) throws IOException { + Utils.createSymlink(dir, targetDir, targetFilename, + targetFilename); + } + + /** + * Returns a Collection of file names found under the given directory. + * @param dir a directory + * @return the Collection of file names + */ + public static Collection readDirContents(String dir) { + Collection ret = new HashSet<>(); + File[] files = new File(dir).listFiles(); + if (files != null) { + for (File f: files) { + ret.add(f.getName()); + } + } + return ret; + } + + /** + * Returns the value of java.class.path System property. Kept separate for + * testing. + * @return the classpath + */ + public static String currentClasspath() { + return _instance.currentClasspathImpl(); + } + + // Non-static impl methods exist for mocking purposes. + public String currentClasspathImpl() { + return System.getProperty("java.class.path"); + } + + /** + * Returns a collection of jar file names found under the given directory. + * @param dir the directory to search + * @return the jar file names + */ + private static List getFullJars(String dir) { + File[] files = new File(dir).listFiles(jarFilter); + + if(files == null) { + return new ArrayList<>(); + } + + List ret = new ArrayList<>(files.length); + for (File f : files) { + ret.add(Paths.get(dir, f.getName()).toString()); + } + return ret; + } + private static final FilenameFilter jarFilter = new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.endsWith(".jar"); + } + }; + + + public static String workerClasspath() { + String stormDir = System.getProperty("storm.home"); + + if (stormDir == null) { + return Utils.currentClasspath(); + } + + String stormLibDir = Paths.get(stormDir, "lib").toString(); + String stormConfDir = + System.getenv("STORM_CONF_DIR") != null ? + System.getenv("STORM_CONF_DIR") : + Paths.get(stormDir, "conf").toString(); + String stormExtlibDir = Paths.get(stormDir, "extlib").toString(); + String extcp = System.getenv("STORM_EXT_CLASSPATH"); + List pathElements = new LinkedList<>(); + pathElements.addAll(Utils.getFullJars(stormLibDir)); + pathElements.addAll(Utils.getFullJars(stormExtlibDir)); + pathElements.add(extcp); + pathElements.add(stormConfDir); + + return StringUtils.join(pathElements, + CLASS_PATH_SEPARATOR); + } + + public static String addToClasspath(String classpath, + Collection paths) { + return _instance.addToClasspathImpl(classpath, paths); + } + + // Non-static impl methods exist for mocking purposes. + public String addToClasspathImpl(String classpath, + Collection paths) { + if (paths == null || paths.isEmpty()) { + return classpath; + } + List l = new LinkedList<>(); + l.add(classpath); + l.addAll(paths); + return StringUtils.join(l, CLASS_PATH_SEPARATOR); + } + + public static class UptimeComputer { + int startTime = 0; + + public UptimeComputer() { + startTime = Time.currentTimeSecs(); + } + + public int upTime() { + return Time.deltaSecs(startTime); + } + } + + public static UptimeComputer makeUptimeComputer() { + return _instance.makeUptimeComputerImpl(); + } + + // Non-static impl methods exist for mocking purposes. + public UptimeComputer makeUptimeComputerImpl() { + return new UptimeComputer(); + } + + /** + * Writes a posix shell script file to be executed in its own process. + * @param dir the directory under which the script is to be written + * @param command the command the script is to execute + * @param environment optional environment variables to set before running the script's command. May be null. + * @return the path to the script that has been written + */ + public static String writeScript(String dir, List command, + Map environment) throws IOException { + String path = Utils.scriptFilePath(dir); + try(BufferedWriter out = new BufferedWriter(new FileWriter(path))) { + out.write("#!/bin/bash"); + out.newLine(); + if (environment != null) { + for (String k : environment.keySet()) { + String v = environment.get(k); + if (v == null) { + v = ""; + } + out.write(Utils.shellCmd( + Arrays.asList( + "export",k+"="+v))); + out.write(";"); + out.newLine(); + } + } + out.newLine(); + out.write("exec "+Utils.shellCmd(command)+";"); + } + return path; + } + + /** + * A thread that can answer if it is sleeping in the case of simulated time. + * This class is not useful when simulated time is not being used. + */ + public static class SmartThread extends Thread { + public boolean isSleeping() { + return Time.isThreadWaiting(this); + } + public SmartThread(Runnable r) { + super(r); + } + } + + /** + * Creates a thread that calls the given code repeatedly, sleeping for an + * interval of seconds equal to the return value of the previous call. + * + * The given afn may be a callable that returns the number of seconds to + * sleep, or it may be a Callable that returns another Callable that in turn + * returns the number of seconds to sleep. In the latter case isFactory. + * + * @param afn the code to call on each iteration + * @param isDaemon whether the new thread should be a daemon thread + * @param eh code to call when afn throws an exception + * @param priority the new thread's priority + * @param isFactory whether afn returns a callable instead of sleep seconds + * @param startImmediately whether to start the thread before returning + * @param threadName a suffix to be appended to the thread name + * @return the newly created thread + * @see java.lang.Thread + */ + public static SmartThread asyncLoop(final Callable afn, + boolean isDaemon, final Thread.UncaughtExceptionHandler eh, + int priority, final boolean isFactory, boolean startImmediately, + String threadName) { + SmartThread thread = new SmartThread(new Runnable() { + public void run() { + Object s; + try { + Callable fn = isFactory ? (Callable) afn.call() : afn; + while ((s = fn.call()) instanceof Long) { + Time.sleepSecs((Long) s); + } + } catch (Throwable t) { + if (Utils.exceptionCauseIsInstanceOf( + InterruptedException.class, t)) { + LOG.info("Async loop interrupted!"); + return; + } + LOG.error("Async loop died!", t); + throw new RuntimeException(t); + } + } + }); + if (eh != null) { + thread.setUncaughtExceptionHandler(eh); + } else { + thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread t, Throwable e) { + LOG.error("Async loop died!", e); + Utils.exitProcess(1, "Async loop died!"); + } + }); + } + thread.setDaemon(isDaemon); + thread.setPriority(priority); + if (threadName != null && !threadName.isEmpty()) { + thread.setName(thread.getName() +"-"+ threadName); + } + if (startImmediately) { + thread.start(); + } + return thread; + } + + /** + * Convenience method used when only the function and name suffix are given. + * @param afn the code to call on each iteration + * @param threadName a suffix to be appended to the thread name + * @return the newly created thread + * @see java.lang.Thread + */ + public static SmartThread asyncLoop(final Callable afn, String threadName, final Thread.UncaughtExceptionHandler eh) { + return asyncLoop(afn, false, eh, Thread.NORM_PRIORITY, false, true, + threadName); + } + + /** + * Convenience method used when only the function is given. + * @param afn the code to call on each iteration + * @return the newly created thread + */ + public static SmartThread asyncLoop(final Callable afn) { + return asyncLoop(afn, false, null, Thread.NORM_PRIORITY, false, true, + null); + } + + /** + * A callback that can accept an integer. + * @param the result type of method call + */ + public interface ExitCodeCallable extends Callable { + V call(int exitCode); + } + + /** + * Launch a new process as per {@link java.lang.ProcessBuilder} with a given + * callback. + * @param command the command to be executed in the new process + * @param environment the environment to be applied to the process. Can be + * null. + * @param logPrefix a prefix for log entries from the output of the process. + * Can be null. + * @param exitCodeCallback code to be called passing the exit code value + * when the process completes + * @param dir the working directory of the new process + * @return the new process + * @throws IOException + * @see java.lang.ProcessBuilder + */ + public static Process launchProcess(List command, + Map environment, + final String logPrefix, + final ExitCodeCallable exitCodeCallback, + File dir) + throws IOException { + return _instance.launchProcessImpl(command, environment, logPrefix, + exitCodeCallback, dir); + } + + public Process launchProcessImpl( + List command, + Map cmdEnv, + final String logPrefix, + final ExitCodeCallable exitCodeCallback, + File dir) + throws IOException { + ProcessBuilder builder = new ProcessBuilder(command); + Map procEnv = builder.environment(); + if (dir != null) { + builder.directory(dir); + } + builder.redirectErrorStream(true); + if (cmdEnv != null) { + procEnv.putAll(cmdEnv); + } + final Process process = builder.start(); + if (logPrefix != null || exitCodeCallback != null) { + Utils.asyncLoop(new Callable() { + public Object call() { + if (logPrefix != null ) { + Utils.readAndLogStream(logPrefix, + process.getInputStream()); + } + if (exitCodeCallback != null) { + try { + process.waitFor(); + } catch (InterruptedException ie) { + LOG.info("{} interrupted", logPrefix); + exitCodeCallback.call(process.exitValue()); + } + } + return null; // Run only once. + } + }); + } + return process; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java index f1c7f323706..c0ebc4d3a2e 100644 --- a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java +++ b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java @@ -57,6 +57,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; +import java.util.Vector; public class Zookeeper { private static Logger LOG = LoggerFactory.getLogger(Zookeeper.class); @@ -395,6 +396,12 @@ public static List tokenizePath(String path) { return rtn; } + public static String parentPath(String path) { + List tokens = tokenizePath(path); + tokens.remove(tokens.size() - 1); + return "/" + StringUtils.join(tokens, "/"); + } + public static String toksToPath(List toks) { StringBuffer buff = new StringBuffer(); buff.append("/"); diff --git a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj index cd2bc266866..5ba66514a70 100644 --- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj +++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj @@ -21,7 +21,7 @@ (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout]) (:import [org.apache.storm.tuple Fields]) - (:use [org.apache.storm testing config clojure util]) + (:use [org.apache.storm testing config clojure]) (:use [org.apache.storm.daemon common]) (:require [org.apache.storm [thrift :as thrift]])) @@ -437,68 +437,50 @@ (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS true TOPOLOGY-KRYO-DECORATORS ["this-is-overriden"]}] - (letlocals - (bind builder (TopologyBuilder.)) - (.setSpout builder "1" (TestPlannerSpout. (Fields. ["conf"]))) - (-> builder - (.setBolt "2" - (TestConfBolt. - {TOPOLOGY-KRYO-DECORATORS ["one" "two"]})) - (.shuffleGrouping "1")) - - (bind results - (complete-topology cluster - (.createTopology builder) - :storm-conf {TOPOLOGY-KRYO-DECORATORS ["one" "three"]} - :mock-sources {"1" [[TOPOLOGY-KRYO-DECORATORS]]})) - (is (= {"topology.kryo.decorators" (list "one" "two" "three")} - (->> (read-tuples results "2") - (apply concat) - (apply hash-map))))))) + (let [builder (TopologyBuilder.) + _ (.setSpout builder "1" (TestPlannerSpout. (Fields. ["conf"]))) + _ (-> builder (.setBolt "2" (TestConfBolt. {TOPOLOGY-KRYO-DECORATORS ["one" "two"]})) (.shuffleGrouping "1")) + results (complete-topology cluster + (.createTopology builder) + :storm-conf {TOPOLOGY-KRYO-DECORATORS ["one" "three"]} + :mock-sources {"1" [[TOPOLOGY-KRYO-DECORATORS]]})] + (is (= {"topology.kryo.decorators" (list "one" "two" "three")} + (->> (read-tuples results "2") (apply concat) (apply hash-map))))))) (deftest test-component-specific-config (with-simulated-time-local-cluster [cluster :daemon-conf {TOPOLOGY-SKIP-MISSING-KRYO-REGISTRATIONS true}] - (letlocals - (bind builder (TopologyBuilder.)) - (.setSpout builder "1" (TestPlannerSpout. (Fields. ["conf"]))) - (-> builder - (.setBolt "2" - (TestConfBolt. - {"fake.config" 123 - TOPOLOGY-MAX-TASK-PARALLELISM 20 - TOPOLOGY-MAX-SPOUT-PENDING 30 - TOPOLOGY-KRYO-REGISTER [{"fake.type" "bad.serializer"} - {"fake.type2" "a.serializer"}] - })) - (.shuffleGrouping "1") - (.setMaxTaskParallelism (int 2)) - (.addConfiguration "fake.config2" 987) - ) - - - (bind results - (complete-topology cluster - (.createTopology builder) - :storm-conf {TOPOLOGY-KRYO-REGISTER [{"fake.type" "good.serializer" "fake.type3" "a.serializer3"}]} - :mock-sources {"1" [["fake.config"] - [TOPOLOGY-MAX-TASK-PARALLELISM] - [TOPOLOGY-MAX-SPOUT-PENDING] - ["fake.config2"] - [TOPOLOGY-KRYO-REGISTER] - ]})) - (is (= {"fake.config" 123 - "fake.config2" 987 - TOPOLOGY-MAX-TASK-PARALLELISM 2 - TOPOLOGY-MAX-SPOUT-PENDING 30 - TOPOLOGY-KRYO-REGISTER {"fake.type" "good.serializer" - "fake.type2" "a.serializer" - "fake.type3" "a.serializer3"}} - (->> (read-tuples results "2") - (apply concat) - (apply hash-map)) - )) - ))) + (let [builder (TopologyBuilder.) + _ (.setSpout builder "1" (TestPlannerSpout. (Fields. ["conf"]))) + _ (-> builder + (.setBolt "2" + (TestConfBolt. + {"fake.config" 123 + TOPOLOGY-MAX-TASK-PARALLELISM 20 + TOPOLOGY-MAX-SPOUT-PENDING 30 + TOPOLOGY-KRYO-REGISTER [{"fake.type" "bad.serializer"} + {"fake.type2" "a.serializer"}]})) + (.shuffleGrouping "1") + (.setMaxTaskParallelism (int 2)) + (.addConfiguration "fake.config2" 987)) + results (complete-topology cluster + (.createTopology builder) + :storm-conf {TOPOLOGY-KRYO-REGISTER [{"fake.type" "good.serializer", "fake.type3" "a.serializer3"}]} + :mock-sources {"1" [["fake.config"] + [TOPOLOGY-MAX-TASK-PARALLELISM] + [TOPOLOGY-MAX-SPOUT-PENDING] + ["fake.config2"] + [TOPOLOGY-KRYO-REGISTER]]})] + (is (= {"fake.config" 123 + "fake.config2" 987 + TOPOLOGY-MAX-TASK-PARALLELISM 2 + TOPOLOGY-MAX-SPOUT-PENDING 30 + TOPOLOGY-KRYO-REGISTER {"fake.type" "good.serializer" + "fake.type2" "a.serializer" + "fake.type3" "a.serializer3"}} + (->> (read-tuples results "2") + (apply concat) + (apply hash-map))))))) (defbolt hooks-bolt ["emit" "ack" "fail" "executed"] {:prepare true} [conf context collector] diff --git a/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj b/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj index cd139d73509..e86e8932c9e 100644 --- a/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj +++ b/storm-core/test/clj/integration/org/apache/storm/testing4j_test.clj @@ -15,7 +15,7 @@ ;; limitations under the License. (ns integration.org.apache.storm.testing4j-test (:use [clojure.test]) - (:use [org.apache.storm config clojure testing util]) + (:use [org.apache.storm config clojure testing]) (:require [integration.org.apache.storm.integration-test :as it]) (:require [org.apache.storm.thrift :as thrift]) (:import [org.apache.storm Testing Config ILocalCluster]) @@ -191,22 +191,19 @@ )))))) (deftest test-test-tuple - (letlocals - ;; test the one-param signature - (bind ^Tuple tuple (Testing/testTuple ["james" "bond"])) - (is (= ["james" "bond"] (.getValues tuple))) - (is (= Utils/DEFAULT_STREAM_ID (.getSourceStreamId tuple))) - (is (= ["field1" "field2"] (-> tuple .getFields .toList))) - (is (= "component" (.getSourceComponent tuple))) - - ;; test the two-params signature - (bind mk-tuple-param (MkTupleParam.)) - (doto mk-tuple-param - (.setStream "test-stream") - (.setComponent "test-component") - (.setFields (into-array String ["fname" "lname"]))) - (bind ^Tuple tuple (Testing/testTuple ["james" "bond"] mk-tuple-param)) - (is (= ["james" "bond"] (.getValues tuple))) - (is (= "test-stream" (.getSourceStreamId tuple))) - (is (= ["fname" "lname"] (-> tuple .getFields .toList))) - (is (= "test-component" (.getSourceComponent tuple))))) + (testing "one-param signature" + (let [tuple (Testing/testTuple ["james" "bond"])] + (is (= ["james" "bond"] (.getValues tuple))) + (is (= Utils/DEFAULT_STREAM_ID (.getSourceStreamId tuple))) + (is (= ["field1" "field2"] (-> tuple .getFields .toList))) + (is (= "component" (.getSourceComponent tuple))))) + (testing "two-params signature" + (let [mk-tuple-param (doto (MkTupleParam.) + (.setStream "test-stream") + (.setComponent "test-component") + (.setFields (into-array String ["fname" "lname"]))) + tuple (Testing/testTuple ["james" "bond"] mk-tuple-param)] + (is (= ["james" "bond"] (.getValues tuple))) + (is (= "test-stream" (.getSourceStreamId tuple))) + (is (= ["fname" "lname"] (-> tuple .getFields .toList))) + (is (= "test-component" (.getSourceComponent tuple)))))) diff --git a/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj index 4c52286d69b..57edb70d1e7 100644 --- a/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj +++ b/storm-core/test/clj/integration/org/apache/storm/trident/integration_test.clj @@ -20,11 +20,22 @@ MemoryMapState$Factory]) (:import [org.apache.storm.trident.state StateSpec]) (:import [org.apache.storm.trident.operation.impl CombinerAggStateUpdater]) - (:use [org.apache.storm.trident testing]) - (:use [org.apache.storm util])) + (:use [org.apache.storm.trident testing])) (bootstrap-imports) +(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)))) + (deftest test-memory-map-get-tuples (t/with-local-cluster [cluster] (with-drpc [drpc] diff --git a/storm-core/test/clj/org/apache/storm/cluster_test.clj b/storm-core/test/clj/org/apache/storm/cluster_test.clj index ffd913e7c07..b146cb078c0 100644 --- a/storm-core/test/clj/org/apache/storm/cluster_test.clj +++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj @@ -22,7 +22,7 @@ (:import [org.mockito Mockito]) (:import [org.mockito.exceptions.base MockitoAssertionError]) (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder]) - (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils]) + (:import [org.apache.storm.utils Time Utils ZookeeperAuthInfo ConfigUtils]) (:import [org.apache.storm.cluster ClusterState]) (:import [org.apache.storm.zookeeper Zookeeper]) (:import [org.apache.storm.testing.staticmocking MockedZookeeper]) @@ -47,6 +47,10 @@ (defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port))) +(defn barr + [& vals] + (byte-array (map byte vals))) + (deftest test-basics (with-inprocess-zookeeper zk-port (let [state (mk-state zk-port)] @@ -177,8 +181,8 @@ assignment2 (Assignment. "/aaa" {} {[2] ["2" 2002]} {} {}) nimbusInfo1 (NimbusInfo. "nimbus1" 6667 false) nimbusInfo2 (NimbusInfo. "nimbus2" 6667 false) - nimbusSummary1 (NimbusSummary. "nimbus1" 6667 (current-time-secs) false "v1") - nimbusSummary2 (NimbusSummary. "nimbus2" 6667 (current-time-secs) false "v2") + nimbusSummary1 (NimbusSummary. "nimbus1" 6667 (Time/currentTimeSecs) false "v1") + nimbusSummary2 (NimbusSummary. "nimbus2" 6667 (Time/currentTimeSecs) false "v2") base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil {}) base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil {})] (is (= [] (.assignments state nil))) @@ -245,17 +249,17 @@ (with-inprocess-zookeeper zk-port (with-simulated-time (let [state (mk-storm-state zk-port)] - (.report-error state "a" "1" (local-hostname) 6700 (RuntimeException.)) + (.report-error state "a" "1" (Utils/localHostname) 6700 (RuntimeException.)) (validate-errors! state "a" "1" ["RuntimeException"]) (advance-time-secs! 1) - (.report-error state "a" "1" (local-hostname) 6700 (IllegalArgumentException.)) + (.report-error state "a" "1" (Utils/localHostname) 6700 (IllegalArgumentException.)) (validate-errors! state "a" "1" ["IllegalArgumentException" "RuntimeException"]) (doseq [i (range 10)] - (.report-error state "a" "2" (local-hostname) 6700 (RuntimeException.)) + (.report-error state "a" "2" (Utils/localHostname) 6700 (RuntimeException.)) (advance-time-secs! 2)) (validate-errors! state "a" "2" (repeat 10 "RuntimeException")) (doseq [i (range 5)] - (.report-error state "a" "2" (local-hostname) 6700 (IllegalArgumentException.)) + (.report-error state "a" "2" (Utils/localHostname) 6700 (IllegalArgumentException.)) (advance-time-secs! 2)) (validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException") (repeat 5 "RuntimeException") @@ -297,7 +301,7 @@ (. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder)) (. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder)) (. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder)) - (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf)) + (Utils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf)) (is (nil? (try (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD)))) diff --git a/storm-core/test/clj/org/apache/storm/drpc_test.clj b/storm-core/test/clj/org/apache/storm/drpc_test.clj index 467c29ad9fc..3dcef7a2c46 100644 --- a/storm-core/test/clj/org/apache/storm/drpc_test.clj +++ b/storm-core/test/clj/org/apache/storm/drpc_test.clj @@ -16,16 +16,15 @@ (ns org.apache.storm.drpc-test (:use [clojure test]) (:import [org.apache.storm.drpc ReturnResults DRPCSpout - LinearDRPCTopologyBuilder] - [org.apache.storm.utils ConfigUtils]) + LinearDRPCTopologyBuilder]) (:import [org.apache.storm.topology FailedException]) (:import [org.apache.storm.coordination CoordinatedBolt$FinishedCallback]) (:import [org.apache.storm LocalDRPC LocalCluster]) (:import [org.apache.storm.tuple Fields]) - (:import [org.apache.storm.utils.ConfigUtils]) + (:import [org.apache.storm.utils ConfigUtils] + [org.apache.storm.utils.staticmocking ConfigUtilsInstaller]) (:import [org.apache.storm.generated DRPCExecutionException]) (:import [java.util.concurrent ConcurrentLinkedQueue]) - (:import [org.apache.storm.testing.staticmocking MockedConfigUtils]) (:use [org.apache.storm config testing clojure]) (:use [org.apache.storm.daemon common drpc]) (:use [conjure core])) @@ -223,9 +222,10 @@ (deftest test-dequeue-req-after-timeout (let [queue (ConcurrentLinkedQueue.) delay-seconds 2 - conf {DRPC-REQUEST-TIMEOUT-SECS delay-seconds}] - (with-open [_ (proxy [MockedConfigUtils] [] - (readStormConfigImpl [] conf))] + conf {DRPC-REQUEST-TIMEOUT-SECS delay-seconds} + mock-cu (proxy [ConfigUtils] [] + (readStormConfigImpl [] conf))] + (with-open [_ (ConfigUtilsInstaller. mock-cu)] (stubbing [acquire-queue queue] (let [drpc-handler (service-handler conf)] (is (thrown? DRPCExecutionException @@ -235,11 +235,12 @@ (deftest test-drpc-timeout-cleanup (let [queue (ConcurrentLinkedQueue.) delay-seconds 1 - conf {DRPC-REQUEST-TIMEOUT-SECS delay-seconds}] - (with-open [_ (proxy [MockedConfigUtils] [] - (readStormConfigImpl [] conf))] + conf {DRPC-REQUEST-TIMEOUT-SECS delay-seconds} + mock-cu (proxy [ConfigUtils] [] + (readStormConfigImpl [] conf))] + (with-open [_ (ConfigUtilsInstaller. mock-cu)] (stubbing [acquire-queue queue - timeout-check-secs delay-seconds] + timeout-check-secs delay-seconds] (let [drpc-handler (service-handler conf)] (is (thrown? DRPCExecutionException (.execute drpc-handler "ArbitraryDRPCFunctionName" "no-args")))))))) diff --git a/storm-core/test/clj/org/apache/storm/logviewer_test.clj b/storm-core/test/clj/org/apache/storm/logviewer_test.clj index c13e869b892..4889c8ea7a4 100644 --- a/storm-core/test/clj/org/apache/storm/logviewer_test.clj +++ b/storm-core/test/clj/org/apache/storm/logviewer_test.clj @@ -20,8 +20,11 @@ (:require [conjure.core]) (:use [clojure test]) (:use [conjure core]) - (:use [org.apache.storm.ui helpers]) - (:import [org.apache.storm.daemon DirectoryCleaner]) + (:use [org.apache.storm testing] + [org.apache.storm.ui helpers]) + (:import [org.apache.storm.daemon DirectoryCleaner] + [org.apache.storm.utils Utils Time] + [org.apache.storm.utils.staticmocking UtilsInstaller]) (:import [java.nio.file Files Path DirectoryStream]) (:import [java.nio.file Files]) (:import [java.nio.file.attribute FileAttribute]) @@ -68,7 +71,7 @@ (deftest test-get-size-for-logdir (testing "get the file sizes of a worker log directory" (stubbing [logviewer/get-stream-for-dir (fn [x] (map #(mk-mock-Path %) (.listFiles x)))] - (let [now-millis (current-time-millis) + (let [now-millis (Time/currentTimeMillis) files1 (into-array File (map #(mk-mock-File {:name (str %) :type :file :mtime (- now-millis (* 100 %)) @@ -82,7 +85,7 @@ (deftest test-mk-FileFilter-for-log-cleanup (testing "log file filter selects the correct worker-log dirs for purge" (stubbing [logviewer/get-stream-for-dir (fn [x] (map #(mk-mock-Path %) (.listFiles x)))] - (let [now-millis (current-time-millis) + (let [now-millis (Time/currentTimeMillis) conf {LOGVIEWER-CLEANUP-AGE-MINS 60 LOGVIEWER-CLEANUP-INTERVAL-SECS 300} cutoff-millis (logviewer/cleanup-cutoff-age-millis conf now-millis) @@ -125,104 +128,106 @@ (deftest test-per-workerdir-cleanup! (testing "cleaner deletes oldest files in each worker dir if files are larger than per-dir quota." - (stubbing [rmr nil] - (let [cleaner (proxy [org.apache.storm.daemon.DirectoryCleaner] [] - (getStreamForDirectory - ([^File dir] - (mk-DirectoryStream - (ArrayList. - (map #(mk-mock-Path %) (.listFiles dir))))))) - now-millis (current-time-millis) - files1 (into-array File (map #(mk-mock-File {:name (str "A" %) - :type :file - :mtime (+ now-millis (* 100 %)) - :length 200 }) - (range 0 10))) - files2 (into-array File (map #(mk-mock-File {:name (str "B" %) - :type :file - :mtime (+ now-millis (* 100 %)) - :length 200 }) - (range 0 10))) - files3 (into-array File (map #(mk-mock-File {:name (str "C" %) - :type :file - :mtime (+ now-millis (* 100 %)) - :length 200 }) - (range 0 10))) - port1-dir (mk-mock-File {:name "/workers-artifacts/topo1/port1" - :type :directory - :files files1}) - port2-dir (mk-mock-File {:name "/workers-artifacts/topo1/port2" - :type :directory - :files files2}) - port3-dir (mk-mock-File {:name "/workers-artifacts/topo2/port3" - :type :directory - :files files3}) - topo1-files (into-array File [port1-dir port2-dir]) - topo2-files (into-array File [port3-dir]) - topo1-dir (mk-mock-File {:name "/workers-artifacts/topo1" - :type :directory - :files topo1-files}) - topo2-dir (mk-mock-File {:name "/workers-artifacts/topo2" - :type :directory - :files topo2-files}) - root-files (into-array File [topo1-dir topo2-dir]) - root-dir (mk-mock-File {:name "/workers-artifacts" - :type :directory - :files root-files}) - deletedFiles (logviewer/per-workerdir-cleanup! root-dir 1200 cleaner)] - (is (= (first deletedFiles) 4)) - (is (= (second deletedFiles) 4)) - (is (= (last deletedFiles) 4)))))) + (with-open [_ (UtilsInstaller. (proxy [Utils] [] + (forceDeleteImpl [path])))] + (let [cleaner (proxy [org.apache.storm.daemon.DirectoryCleaner] [] + (getStreamForDirectory + ([^File dir] + (mk-DirectoryStream + (ArrayList. + (map #(mk-mock-Path %) (.listFiles dir))))))) + now-millis (Time/currentTimeMillis) + files1 (into-array File (map #(mk-mock-File {:name (str "A" %) + :type :file + :mtime (+ now-millis (* 100 %)) + :length 200 }) + (range 0 10))) + files2 (into-array File (map #(mk-mock-File {:name (str "B" %) + :type :file + :mtime (+ now-millis (* 100 %)) + :length 200 }) + (range 0 10))) + files3 (into-array File (map #(mk-mock-File {:name (str "C" %) + :type :file + :mtime (+ now-millis (* 100 %)) + :length 200 }) + (range 0 10))) + port1-dir (mk-mock-File {:name "/workers-artifacts/topo1/port1" + :type :directory + :files files1}) + port2-dir (mk-mock-File {:name "/workers-artifacts/topo1/port2" + :type :directory + :files files2}) + port3-dir (mk-mock-File {:name "/workers-artifacts/topo2/port3" + :type :directory + :files files3}) + topo1-files (into-array File [port1-dir port2-dir]) + topo2-files (into-array File [port3-dir]) + topo1-dir (mk-mock-File {:name "/workers-artifacts/topo1" + :type :directory + :files topo1-files}) + topo2-dir (mk-mock-File {:name "/workers-artifacts/topo2" + :type :directory + :files topo2-files}) + root-files (into-array File [topo1-dir topo2-dir]) + root-dir (mk-mock-File {:name "/workers-artifacts" + :type :directory + :files root-files}) + deletedFiles (logviewer/per-workerdir-cleanup! root-dir 1200 cleaner)] + (is (= (first deletedFiles) 4)) + (is (= (second deletedFiles) 4)) + (is (= (last deletedFiles) 4)))))) (deftest test-global-log-cleanup! (testing "cleaner deletes oldest when files' sizes are larger than the global quota." - (stubbing [rmr nil - logviewer/get-alive-worker-dirs ["/workers-artifacts/topo1/port1"]] - (let [cleaner (proxy [org.apache.storm.daemon.DirectoryCleaner] [] - (getStreamForDirectory - ([^File dir] - (mk-DirectoryStream - (ArrayList. - (map #(mk-mock-Path %) (.listFiles dir))))))) - now-millis (current-time-millis) - files1 (into-array File (map #(mk-mock-File {:name (str "A" % ".log") - :type :file - :mtime (+ now-millis (* 100 %)) - :length 200 }) - (range 0 10))) - files2 (into-array File (map #(mk-mock-File {:name (str "B" %) - :type :file - :mtime (+ now-millis (* 100 %)) - :length 200 }) - (range 0 10))) - files3 (into-array File (map #(mk-mock-File {:name (str "C" %) - :type :file - :mtime (+ now-millis (* 100 %)) - :length 200 }) - (range 0 10))) - port1-dir (mk-mock-File {:name "/workers-artifacts/topo1/port1" - :type :directory - :files files1}) ;; note that port1-dir is active worker containing active logs - port2-dir (mk-mock-File {:name "/workers-artifacts/topo1/port2" - :type :directory - :files files2}) - port3-dir (mk-mock-File {:name "/workers-artifacts/topo2/port3" - :type :directory - :files files3}) - topo1-files (into-array File [port1-dir port2-dir]) - topo2-files (into-array File [port3-dir]) - topo1-dir (mk-mock-File {:name "/workers-artifacts/topo1" - :type :directory - :files topo1-files}) - topo2-dir (mk-mock-File {:name "/workers-artifacts/topo2" - :type :directory - :files topo2-files}) - root-files (into-array File [topo1-dir topo2-dir]) - root-dir (mk-mock-File {:name "/workers-artifacts" - :type :directory - :files root-files}) - deletedFiles (logviewer/global-log-cleanup! root-dir 2400 cleaner)] - (is (= deletedFiles 18)))))) + (stubbing [logviewer/get-alive-worker-dirs ["/workers-artifacts/topo1/port1"]] + (with-open [_ (UtilsInstaller. (proxy [Utils] [] + (forceDeleteImpl [path])))] + (let [cleaner (proxy [org.apache.storm.daemon.DirectoryCleaner] [] + (getStreamForDirectory + ([^File dir] + (mk-DirectoryStream + (ArrayList. + (map #(mk-mock-Path %) (.listFiles dir))))))) + now-millis (Time/currentTimeMillis) + files1 (into-array File (map #(mk-mock-File {:name (str "A" % ".log") + :type :file + :mtime (+ now-millis (* 100 %)) + :length 200 }) + (range 0 10))) + files2 (into-array File (map #(mk-mock-File {:name (str "B" %) + :type :file + :mtime (+ now-millis (* 100 %)) + :length 200 }) + (range 0 10))) + files3 (into-array File (map #(mk-mock-File {:name (str "C" %) + :type :file + :mtime (+ now-millis (* 100 %)) + :length 200 }) + (range 0 10))) + port1-dir (mk-mock-File {:name "/workers-artifacts/topo1/port1" + :type :directory + :files files1}) ;; note that port1-dir is active worker containing active logs + port2-dir (mk-mock-File {:name "/workers-artifacts/topo1/port2" + :type :directory + :files files2}) + port3-dir (mk-mock-File {:name "/workers-artifacts/topo2/port3" + :type :directory + :files files3}) + topo1-files (into-array File [port1-dir port2-dir]) + topo2-files (into-array File [port3-dir]) + topo1-dir (mk-mock-File {:name "/workers-artifacts/topo1" + :type :directory + :files topo1-files}) + topo2-dir (mk-mock-File {:name "/workers-artifacts/topo2" + :type :directory + :files topo2-files}) + root-files (into-array File [topo1-dir topo2-dir]) + root-dir (mk-mock-File {:name "/workers-artifacts" + :type :directory + :files root-files}) + deletedFiles (logviewer/global-log-cleanup! root-dir 2400 cleaner)] + (is (= deletedFiles 18))))))) (deftest test-identify-worker-log-dirs (testing "Build up workerid-workerlogdir map for the old workers' dirs" @@ -252,17 +257,21 @@ (logviewer/get-dead-worker-dirs conf now-secs log-dirs))))))) (deftest test-cleanup-fn - (testing "cleanup function rmr's files of dead workers" + (testing "cleanup function forceDeletes files of dead workers" (let [mockfile1 (mk-mock-File {:name "delete-me1" :type :file}) - mockfile2 (mk-mock-File {:name "delete-me2" :type :file})] - (stubbing [logviewer/select-dirs-for-cleanup nil - logviewer/get-dead-worker-dirs (sorted-set mockfile1 mockfile2) - logviewer/cleanup-empty-topodir! nil - rmr nil] - (logviewer/cleanup-fn! "/bogus/path") - (verify-call-times-for rmr 2) - (verify-nth-call-args-for 1 rmr (.getCanonicalPath mockfile1)) - (verify-nth-call-args-for 2 rmr (.getCanonicalPath mockfile2)))))) + mockfile2 (mk-mock-File {:name "delete-me2" :type :file}) + forceDelete-args (atom []) + utils-proxy (proxy [Utils] [] + (forceDeleteImpl [path] + (swap! forceDelete-args conj path)))] + (with-open [_ (UtilsInstaller. utils-proxy)] + (stubbing [logviewer/select-dirs-for-cleanup nil + logviewer/get-dead-worker-dirs (sorted-set mockfile1 mockfile2) + logviewer/cleanup-empty-topodir! nil] + (logviewer/cleanup-fn! "/bogus/path") + (is (= 2 (count @forceDelete-args))) + (is (= (.getCanonicalPath mockfile1) (get @forceDelete-args 0))) + (is (= (.getCanonicalPath mockfile2) (get @forceDelete-args 1)))))))) (deftest test-authorized-log-user (testing "allow cluster admin" @@ -341,7 +350,7 @@ returned-all (logviewer/list-log-files "user" nil nil root-path nil origin) returned-filter-port (logviewer/list-log-files "user" nil "port1" root-path nil origin) returned-filter-topoId (logviewer/list-log-files "user" "topoB" nil root-path nil origin)] - (rmr root-path) + (Utils/forceDelete root-path) (is (= expected-all returned-all)) (is (= expected-filter-port returned-filter-port)) (is (= expected-filter-topoId returned-filter-topoId))))) @@ -360,23 +369,23 @@ ;; match. exp-offset-fn #(- (/ logviewer/default-bytes-per-page 2) %)] - (stubbing [local-hostname expected-host - logviewer/logviewer-port expected-port] - - (testing "Logviewer link centers the match in the page" - (let [expected-fname "foobar.log"] - (is (= (str "http://" - expected-host - ":" - expected-port - "/log?file=" - expected-fname - "&start=1947&length=" - logviewer/default-bytes-per-page) - (logviewer/url-to-match-centered-in-log-page (byte-array 42) - expected-fname - 27526 - 8888))))) + (stubbing [logviewer/logviewer-port expected-port] + (with-open [_ (UtilsInstaller. (proxy [Utils] [] + (localHostnameImpl [] expected-host)))] + (testing "Logviewer link centers the match in the page" + (let [expected-fname "foobar.log"] + (is (= (str "http://" + expected-host + ":" + expected-port + "/log?file=" + expected-fname + "&start=1947&length=" + logviewer/default-bytes-per-page) + (logviewer/url-to-match-centered-in-log-page (byte-array 42) + expected-fname + 27526 + 8888))))) (let [file (->> "logviewer-search-context-tests.log" (clojure.java.io/file "src" "dev"))] @@ -661,7 +670,7 @@ (logviewer/substring-search file pattern :num-matches nil - :start-byte-offset nil))))))))) + :start-byte-offset nil)))))))))) (deftest test-find-n-matches (testing "find-n-matches looks through logs properly" @@ -761,5 +770,5 @@ ; Called with a bad port (not in the config) No searching should be done. (verify-call-times-for logviewer/find-n-matches 0) (verify-call-times-for logviewer/logs-for-port 0))) - (rmr topo-path)))) + (Utils/forceDelete topo-path)))) diff --git a/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj b/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj index c2b15cef08e..f75a8e3220f 100644 --- a/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj +++ b/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj @@ -17,7 +17,7 @@ (:use [clojure test]) (:import [org.apache.storm.messaging TransportFactory]) (:import [org.apache.storm.testing TestWordSpout TestGlobalCount]) - (:use [org.apache.storm testing util config]) + (:use [org.apache.storm testing config]) (:require [org.apache.storm [thrift :as thrift]])) (deftest test-integration diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj index 19c6f596442..70cb8850a99 100644 --- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj +++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj @@ -24,16 +24,17 @@ (:import [org.apache.storm.testing.staticmocking MockedZookeeper]) (:import [org.apache.storm.scheduler INimbus]) (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo]) - (:import [org.apache.storm.testing.staticmocking MockedConfigUtils]) (:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions InvalidTopologyException AuthorizationException LogConfig LogLevel LogLevelAction]) (:import [java.util HashMap]) (:import [java.io File]) - (:import [org.apache.storm.utils Time Utils ConfigUtils]) + (:import [org.apache.storm.utils Time Utils Utils$UptimeComputer ConfigUtils IPredicate] + [org.apache.storm.utils.staticmocking ConfigUtilsInstaller UtilsInstaller]) (:import [org.apache.storm.zookeeper Zookeeper]) - (:import [org.apache.commons.io FileUtils]) + (:import [org.apache.commons.io FileUtils] + [org.json.simple JSONValue]) (:use [org.apache.storm testing MockAutoCred util config log timer zookeeper]) (:use [org.apache.storm.daemon common]) (:require [conjure.core]) @@ -42,12 +43,20 @@ [cluster :as cluster]]) (:use [conjure core])) +(defn- from-json + [^String str] + (if str + (clojurify-structure + (JSONValue/parse str)) + nil)) + (defn storm-component->task-info [cluster storm-name] (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name) nimbus (:nimbus cluster)] (-> (.getUserTopology nimbus storm-id) (storm-task-info (from-json (.getTopologyConf nimbus storm-id))) - reverse-map))) + (Utils/reverseMap) + clojurify-structure))) (defn getCredentials [cluster storm-name] (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)] @@ -66,12 +75,13 @@ keys (map (fn [e] {e (get-component e)})) (apply merge) - reverse-map))) + (Utils/reverseMap) + clojurify-structure))) (defn storm-num-workers [state storm-name] (let [storm-id (get-storm-id state storm-name) assignment (.assignment-info state storm-id nil)] - (count (reverse-map (:executor->node+port assignment))) + (count (clojurify-structure (Utils/reverseMap (:executor->node+port assignment)))) )) (defn topology-nodes [state storm-name] @@ -93,6 +103,8 @@ set ))) +;TODO: when translating this function, don't call map-val, but instead use an inline for loop. +; map-val is a temporary kluge for clojure. (defn topology-node-distribution [state storm-name] (let [storm-id (get-storm-id state storm-name) assignment (.assignment-info state storm-id nil)] @@ -127,14 +139,13 @@ curr-beat (.get-worker-heartbeat state storm-id node port) stats (:executor-stats curr-beat)] (.worker-heartbeat! state storm-id node port - {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})} + {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})} ))) (defn slot-assignments [cluster storm-id] (let [state (:storm-cluster-state cluster) assignment (.assignment-info state storm-id nil)] - (reverse-map (:executor->node+port assignment)) - )) + (clojurify-structure (Utils/reverseMap (:executor->node+port assignment))))) (defn task-ids [cluster storm-id] (let [nimbus (:nimbus cluster)] @@ -144,14 +155,15 @@ (defn topology-executors [cluster storm-id] (let [state (:storm-cluster-state cluster) - assignment (.assignment-info state storm-id nil)] - (keys (:executor->node+port assignment)) + assignment (.assignment-info state storm-id nil) + ret-keys (keys (:executor->node+port assignment)) + _ (log-message "ret-keys: " (pr-str ret-keys)) ] + ret-keys )) (defn check-distribution [items distribution] - (let [dist (->> items (map count) multi-set)] - (is (= dist (multi-set distribution))) - )) + (let [counts (map count items)] + (is (ms= counts distribution)))) (defn disjoint? [& sets] (let [combined (apply concat sets)] @@ -282,6 +294,18 @@ (is (= (.get (getCredentials cluster topology-name) nimbus-cred-key) nimbus-cred-renew-val)) (is (= (.get (getCredentials cluster topology-name) gateway-cred-key) gateway-cred-renew-val))))) +(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)))) + (deftest test-isolated-assignment (with-simulated-time-local-cluster [cluster :supervisors 6 :ports-per-supervisor 3 @@ -355,6 +379,7 @@ (is (= 2 (storm-num-workers state "mystorm"))) ;; because only 2 executors ))) +;TODO: when translating this function, you should replace the map-val with a proper for loop HERE (deftest test-executor-assignments (with-simulated-time-local-cluster[cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}] (let [nimbus (:nimbus cluster) @@ -613,32 +638,54 @@ (bind [executor-id1 executor-id2] (topology-executors cluster storm-id)) (bind ass1 (executor-assignment cluster storm-id executor-id1)) (bind ass2 (executor-assignment cluster storm-id executor-id2)) + (bind _ (log-message "ass1, t0: " (pr-str ass1))) + (bind _ (log-message "ass2, t0: " (pr-str ass2))) (advance-cluster-time cluster 30) + (bind _ (log-message "ass1, t30, pre beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t30, pre beat: " (pr-str ass2))) (do-executor-heartbeat cluster storm-id executor-id1) (do-executor-heartbeat cluster storm-id executor-id2) + (bind _ (log-message "ass1, t30, post beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t30, post beat: " (pr-str ass2))) (advance-cluster-time cluster 13) + (bind _ (log-message "ass1, t43, pre beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t43, pre beat: " (pr-str ass2))) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (is (= ass2 (executor-assignment cluster storm-id executor-id2))) (do-executor-heartbeat cluster storm-id executor-id1) + (bind _ (log-message "ass1, t43, post beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t43, post beat: " (pr-str ass2))) (advance-cluster-time cluster 11) + (bind _ (log-message "ass1, t54, pre beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t54, pre beat: " (pr-str ass2))) (do-executor-heartbeat cluster storm-id executor-id1) + (bind _ (log-message "ass1, t54, post beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t54, post beat: " (pr-str ass2))) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (check-consistency cluster "test") ; have to wait an extra 10 seconds because nimbus may not ; resynchronize its heartbeat time till monitor-time secs after (advance-cluster-time cluster 11) + (bind _ (log-message "ass1, t65, pre beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t65, pre beat: " (pr-str ass2))) (do-executor-heartbeat cluster storm-id executor-id1) + (bind _ (log-message "ass1, t65, post beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t65, post beat: " (pr-str ass2))) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (check-consistency cluster "test") (advance-cluster-time cluster 11) + (bind _ (log-message "ass1, t76, pre beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t76, pre beat: " (pr-str ass2))) (is (= ass1 (executor-assignment cluster storm-id executor-id1))) (is (not= ass2 (executor-assignment cluster storm-id executor-id2))) (bind ass2 (executor-assignment cluster storm-id executor-id2)) + (bind _ (log-message "ass1, t76, post beat: " (pr-str ass1))) + (bind _ (log-message "ass2, t76, post beat: " (pr-str ass2))) (check-consistency cluster "test") (advance-cluster-time cluster 31) @@ -783,7 +830,8 @@ (check-executor-distribution slot-executors2 [2 2 2 3]) (check-consistency cluster "test") - (bind common (first (find-first (fn [[k v]] (= 3 (count v))) slot-executors2))) + (bind common (first (Utils/findOne (proxy [IPredicate] [] + (test [[k v]] (= 3 (count v)))) slot-executors2))) (is (not-nil? common)) (is (= (slot-executors2 common) (slot-executors common))) @@ -842,6 +890,7 @@ )))) ))) +;TODO: when translating this function, you should replace the map-val with a proper for loop HERE (deftest test-rebalance-change-parallelism (with-simulated-time-local-cluster [cluster :supervisors 4 :ports-per-supervisor 3 :daemon-conf {SUPERVISOR-ENABLE false @@ -1205,21 +1254,23 @@ (let [expected-name topology-name expected-conf {TOPOLOGY-NAME expected-name - :foo :bar}] + "foo" "bar"}] (testing "getTopologyConf calls check-authorization! with the correct parameters." - (let [expected-operation "getTopologyConf"] + (let [expected-operation "getTopologyConf" + expected-conf-json (JSONValue/toJSONString expected-conf)] (stubbing [nimbus/check-authorization! nil - nimbus/try-read-storm-conf expected-conf - util/to-json nil] + nimbus/try-read-storm-conf expected-conf] (try - (.getTopologyConf nimbus "fake-id") + (is (= expected-conf + (->> (.getTopologyConf nimbus "fake-id") + JSONValue/parse + clojurify-structure))) (catch NotAliveException e) (finally (verify-first-call-args-for-indices nimbus/check-authorization! - [1 2 3] expected-name expected-conf expected-operation) - (verify-first-call-args-for util/to-json expected-conf)))))) + [1 2 3] expected-name expected-conf expected-operation)))))) (testing "getTopology calls check-authorization! with the correct parameters." (let [expected-operation "getTopology"] @@ -1347,24 +1398,28 @@ STORM-PRINCIPAL-TO-LOCAL-PLUGIN "org.apache.storm.security.auth.DefaultPrincipalToLocal" NIMBUS-THRIFT-PORT 6666}) expected-acls nimbus/NIMBUS-ZK-ACLS - fake-inimbus (reify INimbus (getForcedScheduler [this] nil))] - (with-open [_ (proxy [MockedConfigUtils] [] + fake-inimbus (reify INimbus (getForcedScheduler [this] nil)) + fake-cu (proxy [ConfigUtils] [] (nimbusTopoHistoryStateImpl [conf] nil)) + fake-utils (proxy [Utils] [] + (newInstanceImpl [_]) + (makeUptimeComputer [] (proxy [Utils$UptimeComputer] [] + (upTime [] 0))))] + (with-open [_ (ConfigUtilsInstaller. fake-cu) + _ (UtilsInstaller. fake-utils) zk-le (MockedZookeeper. (proxy [Zookeeper] [] (zkLeaderElectorImpl [conf] nil)))] (stubbing [mk-authorization-handler nil - cluster/mk-storm-cluster-state nil - nimbus/file-cache-map nil - nimbus/mk-blob-cache-map nil - nimbus/mk-bloblist-cache-map nil - uptime-computer nil - new-instance nil - mk-timer nil - nimbus/mk-scheduler nil] - (nimbus/nimbus-data auth-conf fake-inimbus) - (verify-call-times-for cluster/mk-storm-cluster-state 1) - (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2] - expected-acls)))))) + cluster/mk-storm-cluster-state nil + nimbus/file-cache-map nil + nimbus/mk-blob-cache-map nil + nimbus/mk-bloblist-cache-map nil + mk-timer nil + nimbus/mk-scheduler nil] + (nimbus/nimbus-data auth-conf fake-inimbus) + (verify-call-times-for cluster/mk-storm-cluster-state 1) + (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2] + expected-acls)))))) (deftest test-file-bogus-download (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}] @@ -1397,7 +1452,7 @@ STORM-LOCAL-DIR nimbus-dir})) (bind cluster-state (cluster/mk-storm-cluster-state conf)) (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus))) - (sleep-secs 1) + (Time/sleepSecs 1) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} {})) @@ -1430,7 +1485,7 @@ (bind cluster-state (cluster/mk-storm-cluster-state conf)) (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus))) (bind notifier (InMemoryTopologyActionNotifier.)) - (sleep-secs 1) + (Time/sleepSecs 1) (bind topology (thrift/mk-topology {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)} {})) diff --git a/storm-core/test/clj/org/apache/storm/scheduler/resource_aware_scheduler_test.clj b/storm-core/test/clj/org/apache/storm/scheduler/resource_aware_scheduler_test.clj index ec51914c4a6..f613a5b2e91 100644 --- a/storm-core/test/clj/org/apache/storm/scheduler/resource_aware_scheduler_test.clj +++ b/storm-core/test/clj/org/apache/storm/scheduler/resource_aware_scheduler_test.clj @@ -15,13 +15,14 @@ ;; limitations under the License. (ns org.apache.storm.scheduler.resource-aware-scheduler-test (:use [clojure test]) - (:use [org.apache.storm config testing thrift]) - (:require [org.apache.storm.util :refer [map-val reverse-map sum]]) + (:use [org.apache.storm util config testing thrift]) + (:require [org.apache.storm.util :refer [map-val]]) (:require [org.apache.storm.daemon [nimbus :as nimbus]]) (:import [org.apache.storm.generated StormTopology] [org.apache.storm Config] [org.apache.storm.testing TestWordSpout TestWordCounter] - [org.apache.storm.topology TopologyBuilder]) + [org.apache.storm.topology TopologyBuilder] + [org.apache.storm.utils Utils]) (:import [org.apache.storm.scheduler Cluster SupervisorDetails WorkerSlot ExecutorDetails SchedulerAssignmentImpl Topologies TopologyDetails]) (:import [org.apache.storm.scheduler.resource RAS_Node RAS_Nodes ResourceAwareScheduler]) @@ -54,6 +55,7 @@ (def DEFAULT_SCHEDULING_STRATEGY "org.apache.storm.scheduler.resource.strategies.scheduling.DefaultResourceAwareStrategy") ;; get the super->mem HashMap by counting the eds' mem usage of all topos on each super +;TODO: when translating this function, you should replace the map-val with a proper for loop HERE (defn get-super->mem-usage [^Cluster cluster ^Topologies topologies] (let [assignments (.values (.getAssignments cluster)) supers (.values (.getSupervisors cluster)) @@ -64,7 +66,7 @@ (let [ed->super (into {} (for [[ed slot] (.getExecutorToSlot assignment)] {ed (.getSupervisorById cluster (.getNodeId slot))})) - super->eds (reverse-map ed->super) + super->eds (clojurify-structure (Utils/reverseMap ed->super)) topology (.getById topologies (.getTopologyId assignment)) super->mem-pertopo (map-val (fn [eds] (reduce + (map #(.getTotalMemReqTask topology %) eds))) @@ -75,6 +77,7 @@ super->mem-usage)) ;; get the super->cpu HashMap by counting the eds' cpu usage of all topos on each super +;TODO: when translating this function, you should replace the map-val with a proper for loop HERE (defn get-super->cpu-usage [^Cluster cluster ^Topologies topologies] (let [assignments (.values (.getAssignments cluster)) supers (.values (.getSupervisors cluster)) @@ -85,7 +88,7 @@ (let [ed->super (into {} (for [[ed slot] (.getExecutorToSlot assignment)] {ed (.getSupervisorById cluster (.getNodeId slot))})) - super->eds (reverse-map ed->super) + super->eds (clojurify-structure (Utils/reverseMap ed->super)) topology (.getById topologies (.getTopologyId assignment)) super->cpu-pertopo (map-val (fn [eds] (reduce + (map #(.getTotalCpuReqTask topology %) eds))) @@ -334,13 +337,13 @@ ed->super (into {} (for [[ed slot] (.getExecutorToSlot assignment)] {ed (.getSupervisorById cluster (.getNodeId slot))})) - super->eds (reverse-map ed->super) + super->eds (clojurify-structure (Utils/reverseMap ed->super)) mem-avail->used (into [] (for [[super eds] super->eds] - [(.getTotalMemory super) (sum (map #(.getTotalMemReqTask topology1 %) eds))])) + [(.getTotalMemory super) (reduce + (map #(.getTotalMemReqTask topology1 %) eds))])) cpu-avail->used (into [] (for [[super eds] super->eds] - [(.getTotalCPU super) (sum (map #(.getTotalCpuReqTask topology1 %) eds))]))] + [(.getTotalCPU super) (reduce + (map #(.getTotalCpuReqTask topology1 %) eds))]))] ;; 4 slots on 1 machine, all executors assigned (is (= 2 (.size assigned-slots))) ;; executor0 resides one one worker (on one), executor1 and executor2 on another worker (on the other node) (is (= 2 (.size (into #{} (for [slot assigned-slots] (.getNodeId slot)))))) @@ -403,7 +406,7 @@ assignment (.getAssignmentById cluster "topology2") failed-worker (first (vec (.getSlots assignment))) ;; choose a worker to mock as failed ed->slot (.getExecutorToSlot assignment) - failed-eds (.get (reverse-map ed->slot) failed-worker) + failed-eds (.get (clojurify-structure (Utils/reverseMap ed->slot)) failed-worker) _ (doseq [ed failed-eds] (.remove ed->slot ed)) ;; remove executor details assigned to the worker copy-old-mapping (HashMap. ed->slot) healthy-eds (.keySet copy-old-mapping) diff --git a/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj index 9108f1adcc7..27f5816329b 100644 --- a/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj +++ b/storm-core/test/clj/org/apache/storm/security/auth/auth_test.clj @@ -17,6 +17,8 @@ (:use [clojure test]) (:require [org.apache.storm.daemon [nimbus :as nimbus]]) (:import [org.apache.thrift TException] + [org.json.simple JSONValue] + [org.apache.storm.utils Utils] [org.apache.storm.security.auth.authorizer ImpersonationAuthorizer] [java.net Inet4Address]) (:import [org.apache.thrift.transport TTransportException]) @@ -28,13 +30,14 @@ (:import [org.apache.storm.generated AuthorizationException]) (:import [org.apache.storm.utils NimbusClient ConfigUtils]) (:import [org.apache.storm.security.auth.authorizer SimpleWhitelistAuthorizer SimpleACLAuthorizer]) - (:import [org.apache.storm.security.auth AuthUtils ThriftServer ThriftClient ShellBasedGroupsMapping + (:import [org.apache.storm.security.auth AuthUtils ThriftServer ThriftClient ShellBasedGroupsMapping ReqContext SimpleTransportPlugin KerberosPrincipalToLocal ThriftConnectionType]) (:use [org.apache.storm util config]) (:use [org.apache.storm.daemon common]) (:use [org.apache.storm testing]) (:import [org.apache.storm.generated Nimbus Nimbus$Client Nimbus$Iface StormTopology SubmitOptions - KillOptions RebalanceOptions ClusterSummary TopologyInfo Nimbus$Processor])) + KillOptions RebalanceOptions ClusterSummary TopologyInfo Nimbus$Processor] + (org.json.simple JSONValue))) (defn mk-principal [name] (reify Principal @@ -62,7 +65,7 @@ :heartbeats-cache (atom {}) :downloaders nil :uploaders nil - :uptime (uptime-computer) + :uptime (Utils/makeUptimeComputer) :validator nil :timer nil :scheduler nil @@ -75,7 +78,7 @@ (reify Nimbus$Iface (^void submitTopologyWithOpts [this ^String storm-name ^String uploadedJarLocation ^String serializedConf ^StormTopology topology ^SubmitOptions submitOptions] - (if (not (nil? serializedConf)) (swap! topo-conf (fn [prev new] new) (from-json serializedConf))) + (if (not (nil? serializedConf)) (swap! topo-conf (fn [prev new] new) (if serializedConf (clojurify-structure (JSONValue/parse serializedConf))))) (nimbus/check-authorization! nimbus-d storm-name @topo-conf "submitTopology" auth-context)) (^void killTopology [this ^String storm-name] diff --git a/storm-core/test/clj/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer_test.clj index b18406cd58e..5cce73b6cd0 100644 --- a/storm-core/test/clj/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer_test.clj +++ b/storm-core/test/clj/org/apache/storm/security/auth/authorizer/DRPCSimpleACLAuthorizer_test.clj @@ -19,7 +19,7 @@ (:import [org.apache.storm Config]) (:import [org.apache.storm.security.auth ReqContext SingleUserPrincipal]) (:import [org.apache.storm.security.auth.authorizer DRPCSimpleACLAuthorizer]) - (:use [org.apache.storm config util]) + (:use [org.apache.storm config]) ) (defn- mk-mock-context [user] diff --git a/storm-core/test/clj/org/apache/storm/security/serialization/BlowfishTupleSerializer_test.clj b/storm-core/test/clj/org/apache/storm/security/serialization/BlowfishTupleSerializer_test.clj index deece1b62f2..824e1d89dc2 100644 --- a/storm-core/test/clj/org/apache/storm/security/serialization/BlowfishTupleSerializer_test.clj +++ b/storm-core/test/clj/org/apache/storm/security/serialization/BlowfishTupleSerializer_test.clj @@ -15,7 +15,6 @@ ;; limitations under the License. (ns org.apache.storm.security.serialization.BlowfishTupleSerializer-test (:use [clojure test] - [org.apache.storm.util :only (exception-cause?)] [clojure.string :only (join split)] ) (:import [org.apache.storm.security.serialization BlowfishTupleSerializer] diff --git a/storm-core/test/clj/org/apache/storm/serialization_test.clj b/storm-core/test/clj/org/apache/storm/serialization_test.clj index f8692d920f9..23c45ba28ac 100644 --- a/storm-core/test/clj/org/apache/storm/serialization_test.clj +++ b/storm-core/test/clj/org/apache/storm/serialization_test.clj @@ -59,21 +59,18 @@ ) (deftest test-java-serialization - (letlocals - (bind obj (TestSerObject. 1 2)) - (is (thrown? Exception - (roundtrip [obj] {TOPOLOGY-KRYO-REGISTER {"org.apache.storm.testing.TestSerObject" nil} - TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))) - (is (= [obj] (roundtrip [obj] {TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION true}))))) + (let [obj (TestSerObject. 1 2)] + (is (thrown? Exception + (roundtrip [obj] {TOPOLOGY-KRYO-REGISTER {"org.apache.storm.testing.TestSerObject" nil} + TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))) + (is (= [obj] (roundtrip [obj] {TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION true}))))) (deftest test-kryo-decorator - (letlocals - (bind obj (TestSerObject. 1 2)) - (is (thrown? Exception - (roundtrip [obj] {TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))) - - (is (= [obj] (roundtrip [obj] {TOPOLOGY-KRYO-DECORATORS ["org.apache.storm.testing.TestKryoDecorator"] - TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))))) + (let [obj (TestSerObject. 1 2)] + (is (thrown? Exception + (roundtrip [obj] {TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))) + (is (= [obj] (roundtrip [obj] {TOPOLOGY-KRYO-DECORATORS ["org.apache.storm.testing.TestKryoDecorator"] + TOPOLOGY-FALL-BACK-ON-JAVA-SERIALIZATION false}))))) (defn mk-string [size] (let [builder (StringBuilder.)] diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj index edb161bda37..9c31ddffe8d 100644 --- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj +++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj @@ -21,12 +21,15 @@ (:require [clojure [string :as string] [set :as set]]) (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestPlannerSpout]) (:import [org.apache.storm.scheduler ISupervisor]) - (:import [org.apache.storm.utils ConfigUtils]) + (:import [org.apache.storm.utils Time Utils$UptimeComputer ConfigUtils]) (:import [org.apache.storm.generated RebalanceOptions]) - (:import [org.apache.storm.testing.staticmocking MockedConfigUtils]) + (:import [org.mockito Matchers Mockito]) (:import [java.util UUID]) (:import [java.io File]) (:import [java.nio.file Files]) + (:import [org.apache.storm.utils Utils IPredicate] + [org.apache.storm.utils.staticmocking ConfigUtilsInstaller + UtilsInstaller]) (:import [java.nio.file.attribute FileAttribute]) (:use [org.apache.storm config testing util timer log]) (:use [org.apache.storm.daemon common]) @@ -42,13 +45,15 @@ slot-assigns (for [storm-id (.assignments state nil)] (let [executors (-> (.assignment-info state storm-id nil) :executor->node+port - reverse-map + (Utils/reverseMap) + clojurify-structure (get [supervisor-id port] ))] (when executors [storm-id executors]) )) - ret (find-first not-nil? slot-assigns)] + pred (reify IPredicate (test [this x] (not-nil? x))) + ret (Utils/findOne pred slot-assigns)] (when-not ret - (throw-runtime "Could not find assignment for worker")) + (throw (RuntimeException. "Could not find assignment for worker"))) ret )) @@ -67,6 +72,7 @@ (heartbeat-worker sup p storm-id executors) )))) +;TODO: when translating this function, you should replace the map-val with a proper for loop HERE (defn validate-launched-once [launched supervisor->ports storm-id] (let [counts (map count (vals launched)) launched-supervisor->ports (apply merge-with set/union @@ -78,6 +84,18 @@ (is (= launched-supervisor->ports supervisor->ports)) )) +(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)))) + (deftest launches-assignment (with-simulated-time-local-cluster [cluster :supervisors 0 :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true @@ -230,7 +248,7 @@ (defn check-heartbeat [cluster supervisor-id within-secs] (let [hb (get-heartbeat cluster supervisor-id) time-secs (:time-secs hb) - now (current-time-secs) + now (Time/currentTimeSecs) delta (- now time-secs)] (is (>= delta 0)) (is (<= delta within-secs)) @@ -274,7 +292,7 @@ mock-storm-id "fake-storm-id" mock-worker-id "fake-worker-id" mock-mem-onheap 512 - mock-cp (str file-path-separator "base" class-path-separator file-path-separator "stormjar.jar") + mock-cp (str Utils/FILE_PATH_SEPARATOR "base" Utils/CLASS_PATH_SEPARATOR Utils/FILE_PATH_SEPARATOR "stormjar.jar") mock-sensitivity "S3" mock-cp "/base:/stormjar.jar" exp-args-fn (fn [opts topo-opts classpath] @@ -298,9 +316,9 @@ "-Dworkers.artifacts=/tmp/workers-artifacts" "-Dstorm.conf.file=" "-Dstorm.options=" - (str "-Dstorm.log.dir=" file-path-separator "logs") + (str "-Dstorm.log.dir=" Utils/FILE_PATH_SEPARATOR "logs") (str "-Dlogging.sensitivity=" mock-sensitivity) - (str "-Dlog4j.configurationFile=" file-path-separator "log4j2" file-path-separator "worker.xml") + (str "-Dlog4j.configurationFile=" Utils/FILE_PATH_SEPARATOR "log4j2" Utils/FILE_PATH_SEPARATOR "worker.xml") "-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector" (str "-Dstorm.id=" mock-storm-id) (str "-Dworker.id=" mock-worker-id) @@ -319,26 +337,35 @@ mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed WORKER-CHILDOPTS string-opts}} mocked-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS - topo-string-opts}] - (with-open [_ (proxy [MockedConfigUtils] [] + topo-string-opts} + utils-spy (->> + (proxy [Utils] [] + (addToClasspathImpl [classpath paths] mock-cp) + (launchProcessImpl [& _] nil)) + Mockito/spy) + cu-proxy (proxy [ConfigUtils] [] (supervisorStormDistRootImpl ([conf] nil) ([conf storm-id] nil)) (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) (setWorkerUserWSEImpl [conf worker-id user] nil) (workerArtifactsRootImpl [conf] "/tmp/workers-artifacts"))] - (stubbing [add-to-classpath mock-cp - launch-process nil - supervisor/jlp nil - supervisor/write-log-metadata! nil - supervisor/create-blobstore-links nil] + (with-open [_ (ConfigUtilsInstaller. cu-proxy) + _ (UtilsInstaller. utils-spy)] + (stubbing [supervisor/jlp nil + supervisor/write-log-metadata! nil + supervisor/create-blobstore-links nil] (supervisor/launch-worker mock-supervisor mock-storm-id mock-port mock-worker-id mock-mem-onheap) - (verify-first-call-args-for-indices launch-process - [0] - exp-args))))) + (. (Mockito/verify utils-spy) + (launchProcessImpl (Matchers/eq exp-args) + (Matchers/any) + (Matchers/any) + (Matchers/any) + (Matchers/any))))))) + (testing "testing *.worker.childopts as list of strings, with spaces in values" (let [list-opts '("-Dopt1='this has a space in it'" "-Xmx1024m") topo-list-opts '("-Dopt2='val with spaces'" "-Xmx2048m") @@ -346,75 +373,102 @@ mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed WORKER-CHILDOPTS list-opts}} mocked-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS - topo-list-opts}] - (with-open [_ (proxy [MockedConfigUtils] [] - (supervisorStormDistRootImpl ([conf] nil) - ([conf storm-id] nil)) - (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) - (setWorkerUserWSEImpl [conf worker-id user] nil) - (workerArtifactsRootImpl [conf] "/tmp/workers-artifacts"))] - (stubbing [add-to-classpath mock-cp - launch-process nil - supervisor/jlp nil - supervisor/write-log-metadata! nil - supervisor/create-blobstore-links nil] - (supervisor/launch-worker mock-supervisor - mock-storm-id - mock-port - mock-worker-id - mock-mem-onheap) - (verify-first-call-args-for-indices launch-process - [0] - exp-args))))) + topo-list-opts} + cu-proxy (proxy [ConfigUtils] [] + (supervisorStormDistRootImpl ([conf] nil) + ([conf storm-id] nil)) + (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) + (setWorkerUserWSEImpl [conf worker-id user] nil) + (workerArtifactsRootImpl [conf] "/tmp/workers-artifacts")) + utils-spy (->> + (proxy [Utils] [] + (addToClasspathImpl [classpath paths] mock-cp) + (launchProcessImpl [& _] nil)) + Mockito/spy)] + (with-open [_ (ConfigUtilsInstaller. cu-proxy) + _ (UtilsInstaller. utils-spy)] + (stubbing [supervisor/jlp nil + supervisor/write-log-metadata! nil + supervisor/create-blobstore-links nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id + mock-mem-onheap) + (. (Mockito/verify utils-spy) + (launchProcessImpl (Matchers/eq exp-args) + (Matchers/any) + (Matchers/any) + (Matchers/any) + (Matchers/any))))))) + (testing "testing topology.classpath is added to classpath" - (let [topo-cp (str file-path-separator "any" file-path-separator "path") - exp-args (exp-args-fn [] [] (add-to-classpath mock-cp [topo-cp])) + (let [topo-cp (str Utils/FILE_PATH_SEPARATOR "any" Utils/FILE_PATH_SEPARATOR "path") + exp-args (exp-args-fn [] [] (Utils/addToClasspath mock-cp [topo-cp])) mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed}} - mocked-supervisor-storm-conf {TOPOLOGY-CLASSPATH topo-cp}] - (with-open [_ (proxy [MockedConfigUtils] [] + mocked-supervisor-storm-conf {TOPOLOGY-CLASSPATH topo-cp} + cu-proxy (proxy [ConfigUtils] [] (supervisorStormDistRootImpl ([conf] nil) ([conf storm-id] nil)) (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) (setWorkerUserWSEImpl [conf worker-id user] nil) - (workerArtifactsRootImpl [conf] "/tmp/workers-artifacts"))] + (workerArtifactsRootImpl [conf] "/tmp/workers-artifacts")) + utils-spy (->> + (proxy [Utils] [] + (currentClasspathImpl [] + (str Utils/FILE_PATH_SEPARATOR "base")) + (launchProcessImpl [& _] nil)) + Mockito/spy)] + (with-open [_ (ConfigUtilsInstaller. cu-proxy) + _ (UtilsInstaller. utils-spy)] (stubbing [supervisor/jlp nil supervisor/write-log-metadata! nil - launch-process nil - current-classpath (str file-path-separator "base") supervisor/create-blobstore-links nil] - (supervisor/launch-worker mock-supervisor + (supervisor/launch-worker mock-supervisor mock-storm-id mock-port mock-worker-id mock-mem-onheap) - (verify-first-call-args-for-indices launch-process - [0] - exp-args))))) + (. (Mockito/verify utils-spy) + (launchProcessImpl (Matchers/eq exp-args) + (Matchers/any) + (Matchers/any) + (Matchers/any) + (Matchers/any))))))) (testing "testing topology.environment is added to environment for worker launch" (let [topo-env {"THISVAR" "somevalue" "THATVAR" "someothervalue"} full-env (merge topo-env {"LD_LIBRARY_PATH" nil}) exp-args (exp-args-fn [] [] mock-cp) mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed}} - mocked-supervisor-storm-conf {TOPOLOGY-ENVIRONMENT topo-env}] - (with-open [_ (proxy [MockedConfigUtils] [] + mocked-supervisor-storm-conf {TOPOLOGY-ENVIRONMENT topo-env} + cu-proxy (proxy [ConfigUtils] [] (supervisorStormDistRootImpl ([conf] nil) ([conf storm-id] nil)) (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) (setWorkerUserWSEImpl [conf worker-id user] nil) - (workerArtifactsRootImpl [conf] "/tmp/workers-artifacts"))] + (workerArtifactsRootImpl [conf] "/tmp/workers-artifacts")) + utils-spy (->> + (proxy [Utils] [] + (currentClasspathImpl [] + (str Utils/FILE_PATH_SEPARATOR "base")) + (launchProcessImpl [& _] nil)) + Mockito/spy)] + (with-open [_ (ConfigUtilsInstaller. cu-proxy) + _ (UtilsInstaller. utils-spy)] (stubbing [supervisor/jlp nil - launch-process nil - supervisor/write-log-metadata! nil - current-classpath (str file-path-separator "base") - supervisor/create-blobstore-links nil] - (supervisor/launch-worker mock-supervisor - mock-storm-id - mock-port - mock-worker-id - mock-mem-onheap) - (verify-first-call-args-for-indices launch-process - [2] - full-env)))))))) + supervisor/write-log-metadata! nil + supervisor/create-blobstore-links nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id + mock-mem-onheap) + (. (Mockito/verify utils-spy) + (launchProcessImpl (Matchers/any) + (Matchers/eq full-env) + (Matchers/any) + (Matchers/any) + (Matchers/any)))))))))) (deftest test-worker-launch-command-run-as-user (testing "*.worker.childopts configuration" @@ -446,8 +500,8 @@ " '-DLog4jContextSelector=org.apache.logging.log4j.core.selector.BasicContextSelector'" " 'org.apache.storm.LogWriter'" " 'java' '-server'" - " " (shell-cmd opts) - " " (shell-cmd topo-opts) + " " (Utils/shellCmd opts) + " " (Utils/shellCmd topo-opts) " '-Djava.library.path='" " '-Dlogfile.name=" "worker.log'" " '-Dstorm.home='" @@ -480,27 +534,35 @@ WORKER-CHILDOPTS string-opts}} mocked-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS topo-string-opts - TOPOLOGY-SUBMITTER-USER "me"}] - (with-open [_ (proxy [MockedConfigUtils] [] - (supervisorStormDistRootImpl ([conf] nil) - ([conf storm-id] nil)) - (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) - (setWorkerUserWSEImpl [conf worker-id user] nil))] - (stubbing [add-to-classpath mock-cp - launch-process nil - supervisor/java-cmd "java" - supervisor/jlp nil - supervisor/write-log-metadata! nil] - (supervisor/launch-worker mock-supervisor - mock-storm-id - mock-port - mock-worker-id - mock-mem-onheap) - (verify-first-call-args-for-indices launch-process - [0] - exp-launch))) + TOPOLOGY-SUBMITTER-USER "me"} + cu-proxy (proxy [ConfigUtils] [] + (supervisorStormDistRootImpl ([conf] nil) + ([conf storm-id] nil)) + (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) + (setWorkerUserWSEImpl [conf worker-id user] nil)) + utils-spy (->> + (proxy [Utils] [] + (addToClasspathImpl [classpath paths] mock-cp) + (launchProcessImpl [& _] nil)) + Mockito/spy)] + (with-open [_ (ConfigUtilsInstaller. cu-proxy) + _ (UtilsInstaller. utils-spy)] + (stubbing [supervisor/java-cmd "java" + supervisor/jlp nil + supervisor/write-log-metadata! nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id + mock-mem-onheap) + (. (Mockito/verify utils-spy) + (launchProcessImpl (Matchers/eq exp-launch) + (Matchers/any) + (Matchers/any) + (Matchers/any) + (Matchers/any))))) (is (= (slurp worker-script) exp-script)))) - (finally (rmr storm-local))) + (finally (Utils/forceDelete storm-local))) (.mkdirs (io/file storm-local "workers" mock-worker-id)) (try (testing "testing *.worker.childopts as list of strings, with spaces in values" @@ -514,27 +576,35 @@ WORKER-CHILDOPTS list-opts}} mocked-supervisor-storm-conf {TOPOLOGY-WORKER-CHILDOPTS topo-list-opts - TOPOLOGY-SUBMITTER-USER "me"}] - (with-open [_ (proxy [MockedConfigUtils] [] - (supervisorStormDistRootImpl ([conf] nil) - ([conf storm-id] nil)) - (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) - (setWorkerUserWSEImpl [conf worker-id user] nil))] - (stubbing [add-to-classpath mock-cp - launch-process nil - supervisor/java-cmd "java" - supervisor/jlp nil - supervisor/write-log-metadata! nil] - (supervisor/launch-worker mock-supervisor - mock-storm-id - mock-port - mock-worker-id - mock-mem-onheap) - (verify-first-call-args-for-indices launch-process - [0] - exp-launch))) + TOPOLOGY-SUBMITTER-USER "me"} + cu-proxy (proxy [ConfigUtils] [] + (supervisorStormDistRootImpl ([conf] nil) + ([conf storm-id] nil)) + (readSupervisorStormConfImpl [conf storm-id] mocked-supervisor-storm-conf) + (setWorkerUserWSEImpl [conf worker-id user] nil)) + utils-spy (->> + (proxy [Utils] [] + (addToClasspathImpl [classpath paths] mock-cp) + (launchProcessImpl [& _] nil)) + Mockito/spy)] + (with-open [_ (ConfigUtilsInstaller. cu-proxy) + _ (UtilsInstaller. utils-spy)] + (stubbing [supervisor/java-cmd "java" + supervisor/jlp nil + supervisor/write-log-metadata! nil] + (supervisor/launch-worker mock-supervisor + mock-storm-id + mock-port + mock-worker-id + mock-mem-onheap) + (. (Mockito/verify utils-spy) + (launchProcessImpl (Matchers/eq exp-launch) + (Matchers/any) + (Matchers/any) + (Matchers/any) + (Matchers/any))))) (is (= (slurp worker-script) exp-script)))) - (finally (rmr storm-local)))))) + (finally (Utils/forceDelete storm-local)))))) (deftest test-workers-go-bananas ;; test that multiple workers are started for a port, and test that @@ -561,195 +631,204 @@ expected-acls supervisor/SUPERVISOR-ZK-ACLS fake-isupervisor (reify ISupervisor (getSupervisorId [this] nil) - (getAssignmentId [this] nil))] - (with-open [_ (proxy [MockedConfigUtils] [] - (supervisorStateImpl [conf] nil) - (supervisorLocalDirImpl [conf] nil))] - (stubbing [uptime-computer nil - cluster/mk-storm-cluster-state nil - local-hostname nil - mk-timer nil] + (getAssignmentId [this] nil)) + fake-cu (proxy [ConfigUtils] [] + (supervisorStateImpl [conf] nil) + (supervisorLocalDirImpl [conf] nil)) + fake-utils (proxy [Utils] [] + (localHostnameImpl [] nil) + (makeUptimeComputer [] (proxy [Utils$UptimeComputer] [] + (upTime [] 0))))] + (with-open [_ (ConfigUtilsInstaller. fake-cu) + _ (UtilsInstaller. fake-utils)] + (stubbing [cluster/mk-storm-cluster-state nil + mk-timer nil] (supervisor/supervisor-data auth-conf nil fake-isupervisor) (verify-call-times-for cluster/mk-storm-cluster-state 1) (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2] - expected-acls)))))) + expected-acls))))) -(deftest test-write-log-metadata - (testing "supervisor writes correct data to logs metadata file" - (let [exp-owner "alice" - exp-worker-id "42" - exp-storm-id "0123456789" - exp-port 4242 - exp-logs-users ["bob" "charlie" "daryl"] - exp-logs-groups ["read-only-group" "special-group"] - storm-conf {TOPOLOGY-SUBMITTER-USER "alice" - TOPOLOGY-USERS ["charlie" "bob"] - TOPOLOGY-GROUPS ["special-group"] - LOGS-GROUPS ["read-only-group"] - LOGS-USERS ["daryl"]} - exp-data {TOPOLOGY-SUBMITTER-USER exp-owner - "worker-id" exp-worker-id - LOGS-USERS exp-logs-users - LOGS-GROUPS exp-logs-groups} - conf {}] - (mocking [supervisor/write-log-metadata-to-yaml-file!] - (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id - exp-storm-id exp-port conf) - (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file! - exp-storm-id exp-port exp-data conf))))) + (deftest test-write-log-metadata + (testing "supervisor writes correct data to logs metadata file" + (let [exp-owner "alice" + exp-worker-id "42" + exp-storm-id "0123456789" + exp-port 4242 + exp-logs-users ["bob" "charlie" "daryl"] + exp-logs-groups ["read-only-group" "special-group"] + storm-conf {TOPOLOGY-SUBMITTER-USER "alice" + TOPOLOGY-USERS ["charlie" "bob"] + TOPOLOGY-GROUPS ["special-group"] + LOGS-GROUPS ["read-only-group"] + LOGS-USERS ["daryl"]} + exp-data {TOPOLOGY-SUBMITTER-USER exp-owner + "worker-id" exp-worker-id + LOGS-USERS exp-logs-users + LOGS-GROUPS exp-logs-groups} + conf {}] + (mocking [supervisor/write-log-metadata-to-yaml-file!] + (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id + exp-storm-id exp-port conf) + (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file! + exp-storm-id exp-port exp-data conf))))) -(deftest test-worker-launcher-requires-user - (testing "worker-launcher throws on blank user" - (mocking [launch-process] - (is (thrown-cause-with-msg? java.lang.IllegalArgumentException - #"(?i).*user cannot be blank.*" - (supervisor/worker-launcher {} nil "")))))) + (deftest test-worker-launcher-requires-user + (testing "worker-launcher throws on blank user" + (let [utils-proxy (proxy [Utils] [] + (launchProcessImpl [& _] nil))] + (with-open [_ (UtilsInstaller. utils-proxy)] + (is (try + (supervisor/worker-launcher {} nil "") + false + (catch Throwable t + (and (re-matches #"(?i).*user cannot be blank.*" (.getMessage t)) + (Utils/exceptionCauseIsInstanceOf java.lang.IllegalArgumentException t))))))))) -(defn found? [sub-str input-str] - (if (string? input-str) - (contrib-str/substring? sub-str (str input-str)) - (boolean (some #(contrib-str/substring? sub-str %) input-str)))) + (defn found? [sub-str input-str] + (if (string? input-str) + (contrib-str/substring? sub-str (str input-str)) + (boolean (some #(contrib-str/substring? sub-str %) input-str)))) -(defn not-found? [sub-str input-str] + (defn not-found? [sub-str input-str] (complement (found? sub-str input-str))) -(deftest test-substitute-childopts-happy-path-string - (testing "worker-launcher replaces ids in childopts" - (let [worker-id "w-01" - topology-id "s-01" - port 9999 - mem-onheap 512 - childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m" - expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m") - childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] - (is (= expected-childopts childopts-with-ids))))) + (deftest test-substitute-childopts-happy-path-string + (testing "worker-launcher replaces ids in childopts" + (let [worker-id "w-01" + topology-id "s-01" + port 9999 + mem-onheap 512 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m" + expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m") + childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] + (is (= expected-childopts childopts-with-ids))))) -(deftest test-substitute-childopts-happy-path-list - (testing "worker-launcher replaces ids in childopts" - (let [worker-id "w-01" - topology-id "s-01" - port 9999 - mem-onheap 512 - childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m") - expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m") - childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] - (is (= expected-childopts childopts-with-ids))))) + (deftest test-substitute-childopts-happy-path-list + (testing "worker-launcher replaces ids in childopts" + (let [worker-id "w-01" + topology-id "s-01" + port 9999 + mem-onheap 512 + childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m") + expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m") + childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] + (is (= expected-childopts childopts-with-ids))))) -(deftest test-substitute-childopts-happy-path-list-arraylist - (testing "worker-launcher replaces ids in childopts" - (let [worker-id "w-01" - topology-id "s-01" - port 9999 - mem-onheap 512 - childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"] - expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m") - childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] - (is (= expected-childopts childopts-with-ids))))) + (deftest test-substitute-childopts-happy-path-list-arraylist + (testing "worker-launcher replaces ids in childopts" + (let [worker-id "w-01" + topology-id "s-01" + port 9999 + mem-onheap 512 + childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"] + expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m") + childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] + (is (= expected-childopts childopts-with-ids))))) -(deftest test-substitute-childopts-topology-id-alone - (testing "worker-launcher replaces ids in childopts" - (let [worker-id "w-01" - topology-id "s-01" - port 9999 - mem-onheap 512 - childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log" - expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log") - childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] - (is (= expected-childopts childopts-with-ids))))) + (deftest test-substitute-childopts-topology-id-alone + (testing "worker-launcher replaces ids in childopts" + (let [worker-id "w-01" + topology-id "s-01" + port 9999 + mem-onheap 512 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log" + expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log") + childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] + (is (= expected-childopts childopts-with-ids))))) -(deftest test-substitute-childopts-no-keys - (testing "worker-launcher has no ids to replace in childopts" - (let [worker-id "w-01" - topology-id "s-01" - port 9999 - mem-onheap 512 - childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log" - expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log") - childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] - (is (= expected-childopts childopts-with-ids))))) + (deftest test-substitute-childopts-no-keys + (testing "worker-launcher has no ids to replace in childopts" + (let [worker-id "w-01" + topology-id "s-01" + port 9999 + mem-onheap 512 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log" + expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log") + childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] + (is (= expected-childopts childopts-with-ids))))) -(deftest test-substitute-childopts-nil-childopts - (testing "worker-launcher has nil childopts" - (let [worker-id "w-01" - topology-id "s-01" - port 9999 - mem-onheap 512 - childopts nil - expected-childopts nil - childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] - (is (= expected-childopts childopts-with-ids))))) + (deftest test-substitute-childopts-nil-childopts + (testing "worker-launcher has nil childopts" + (let [worker-id "w-01" + topology-id "s-01" + port 9999 + mem-onheap 512 + childopts nil + expected-childopts nil + childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] + (is (= expected-childopts childopts-with-ids))))) -(deftest test-substitute-childopts-nil-ids - (testing "worker-launcher has nil ids" - (let [worker-id nil - topology-id "s-01" - port 9999 - mem-onheap 512 - childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" - expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log") - childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] - (is (= expected-childopts childopts-with-ids))))) + (deftest test-substitute-childopts-nil-ids + (testing "worker-launcher has nil ids" + (let [worker-id nil + topology-id "s-01" + port 9999 + mem-onheap 512 + childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" + expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log") + childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)] + (is (= expected-childopts childopts-with-ids))))) -(deftest test-retry-read-assignments - (with-simulated-time-local-cluster [cluster - :supervisors 0 - :ports-per-supervisor 2 - :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true - NIMBUS-MONITOR-FREQ-SECS 10 - TOPOLOGY-MESSAGE-TIMEOUT-SECS 30 - TOPOLOGY-ACKER-EXECUTORS 0}] - (letlocals - (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4])) - (bind topology1 (thrift/mk-topology - {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)} - {})) - (bind topology2 (thrift/mk-topology - {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)} - {})) - (bind state (:storm-cluster-state cluster)) - (bind changed (capture-changed-workers - (submit-mocked-assignment - (:nimbus cluster) - (:storm-cluster-state cluster) - "topology1" - {TOPOLOGY-WORKERS 2} - topology1 - {1 "1" - 2 "1"} - {[1 1] ["sup1" 1] - [2 2] ["sup1" 2]} - {["sup1" 1] [0.0 0.0 0.0] - ["sup1" 2] [0.0 0.0 0.0] - }) - (submit-mocked-assignment - (:nimbus cluster) - (:storm-cluster-state cluster) - "topology2" - {TOPOLOGY-WORKERS 2} - topology2 - {1 "1" - 2 "1"} - {[1 1] ["sup1" 1] - [2 2] ["sup1" 2]} - {["sup1" 1] [0.0 0.0 0.0] - ["sup1" 2] [0.0 0.0 0.0] - }) - ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called. - (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0))) - )) - (is (empty? (:launched changed))) - (bind options (RebalanceOptions.)) - (.set_wait_secs options 0) - (bind changed (capture-changed-workers - (.rebalance (:nimbus cluster) "topology2" options) - (advance-cluster-time cluster 10) - (heartbeat-workers cluster "sup1" [1 2 3 4]) - (advance-cluster-time cluster 10) - )) - (validate-launched-once (:launched changed) - {"sup1" [1 2]} - (get-storm-id (:storm-cluster-state cluster) "topology1")) - (validate-launched-once (:launched changed) - {"sup1" [3 4]} - (get-storm-id (:storm-cluster-state cluster) "topology2")) - ))) \ No newline at end of file + (deftest test-retry-read-assignments + (with-simulated-time-local-cluster [cluster + :supervisors 0 + :ports-per-supervisor 2 + :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true + NIMBUS-MONITOR-FREQ-SECS 10 + TOPOLOGY-MESSAGE-TIMEOUT-SECS 30 + TOPOLOGY-ACKER-EXECUTORS 0}] + (letlocals + (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4])) + (bind topology1 (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)} + {})) + (bind topology2 (thrift/mk-topology + {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)} + {})) + (bind state (:storm-cluster-state cluster)) + (bind changed (capture-changed-workers + (submit-mocked-assignment + (:nimbus cluster) + (:storm-cluster-state cluster) + "topology1" + {TOPOLOGY-WORKERS 2} + topology1 + {1 "1" + 2 "1"} + {[1 1] ["sup1" 1] + [2 2] ["sup1" 2]} + {["sup1" 1] [0.0 0.0 0.0] + ["sup1" 2] [0.0 0.0 0.0] + }) + (submit-mocked-assignment + (:nimbus cluster) + (:storm-cluster-state cluster) + "topology2" + {TOPOLOGY-WORKERS 2} + topology2 + {1 "1" + 2 "1"} + {[1 1] ["sup1" 1] + [2 2] ["sup1" 2]} + {["sup1" 1] [0.0 0.0 0.0] + ["sup1" 2] [0.0 0.0 0.0] + }) + ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called. + (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0))) + )) + (is (empty? (:launched changed))) + (bind options (RebalanceOptions.)) + (.set_wait_secs options 0) + (bind changed (capture-changed-workers + (.rebalance (:nimbus cluster) "topology2" options) + (advance-cluster-time cluster 10) + (heartbeat-workers cluster "sup1" [1 2 3 4]) + (advance-cluster-time cluster 10) + )) + (validate-launched-once (:launched changed) + {"sup1" [1 2]} + (get-storm-id (:storm-cluster-state cluster) "topology1")) + (validate-launched-once (:launched changed) + {"sup1" [3 4]} + (get-storm-id (:storm-cluster-state cluster) "topology2")) + )))) diff --git a/storm-core/test/clj/org/apache/storm/transactional_test.clj b/storm-core/test/clj/org/apache/storm/transactional_test.clj index 255128bbd6f..dd46a7d63ff 100644 --- a/storm-core/test/clj/org/apache/storm/transactional_test.clj +++ b/storm-core/test/clj/org/apache/storm/transactional_test.clj @@ -72,6 +72,7 @@ (defn normalize-tx-tuple [values] (-> values vec (update 0 #(-> % .getTransactionId .intValue)))) +;TODO: when translating this function, you should replace the map-val with a proper for loop HERE (defn verify-and-reset! [expected-map emitted-map-atom] (let [results @emitted-map-atom] (dorun @@ -99,6 +100,18 @@ (defn get-commit [capture-atom] (-> @capture-atom (get COMMIT-STREAM) first :id)) +(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)))) + (deftest test-coordinator (let [coordinator-state (atom nil) emit-capture (atom nil)] @@ -345,6 +358,11 @@ (RegisteredGlobalState/clearState id#) )) +(defn separate + [pred aseq] + [(filter pred aseq) (filter (complement pred) aseq)]) + + (deftest test-transactional-topology (with-tracked-cluster [cluster] (with-controller-bolt [controller collector tuples] diff --git a/storm-core/test/clj/org/apache/storm/trident/state_test.clj b/storm-core/test/clj/org/apache/storm/trident/state_test.clj index 58a0cb8a5e2..5f31175b583 100644 --- a/storm-core/test/clj/org/apache/storm/trident/state_test.clj +++ b/storm-core/test/clj/org/apache/storm/trident/state_test.clj @@ -18,6 +18,7 @@ (:require [org.apache.storm [testing :as t]]) (:import [org.apache.storm.trident.operation.builtin Count]) (:import [org.apache.storm.trident.state OpaqueValue]) + (:import [org.apache.storm.utils Utils]) (:import [org.apache.storm.trident.state CombinerValueUpdater]) (:import [org.apache.storm.trident.topology.state TransactionalState TestTransactionalState]) (:import [org.apache.storm.trident.state.map TransactionalMap OpaqueMap]) @@ -29,7 +30,7 @@ (:import [org.mockito Matchers Mockito]) (:import [org.mockito.exceptions.base MockitoAssertionError]) (:use [org.apache.storm.trident testing]) - (:use [org.apache.storm config util])) + (:use [org.apache.storm config])) (defn single-remove [map key] (-> map (.multiRemove [[key]]))) @@ -128,7 +129,7 @@ e))))))) (deftest test-memory-map-state-remove - (let [map (MemoryMapState. (uuid))] + (let [map (MemoryMapState. (Utils/uuid))] (.beginCommit map 1) (single-put map "a" 1) (single-put map "b" 2) diff --git a/storm-core/test/clj/org/apache/storm/trident/tuple_test.clj b/storm-core/test/clj/org/apache/storm/trident/tuple_test.clj index b5a811654a5..0971afcd563 100644 --- a/storm-core/test/clj/org/apache/storm/trident/tuple_test.clj +++ b/storm-core/test/clj/org/apache/storm/trident/tuple_test.clj @@ -19,8 +19,19 @@ (:import [org.apache.storm.trident.tuple TridentTupleView TridentTupleView$ProjectionFactory TridentTupleView$FreshOutputFactory TridentTupleView$OperationOutputFactory TridentTupleView$RootFactory]) - (:use [org.apache.storm.trident testing]) - (:use [org.apache.storm util])) + (:use [org.apache.storm.trident testing])) + +(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)))) (deftest test-fresh (letlocals diff --git a/storm-core/test/clj/org/apache/storm/utils_test.clj b/storm-core/test/clj/org/apache/storm/utils_test.clj index cc78e74a2f7..26442aa98c7 100644 --- a/storm-core/test/clj/org/apache/storm/utils_test.clj +++ b/storm-core/test/clj/org/apache/storm/utils_test.clj @@ -18,7 +18,7 @@ (:import [org.apache.storm.utils NimbusClient Utils]) (:import [org.apache.curator.retry ExponentialBackoffRetry]) (:import [org.apache.thrift.transport TTransportException]) - (:import [org.apache.storm.utils ConfigUtils]) + (:import [org.apache.storm.utils ConfigUtils Time]) (:use [org.apache.storm config util]) (:use [clojure test]) ) @@ -100,12 +100,12 @@ (.remove (System/getProperties) k)))))) (deftest test-secs-to-millis-long - (is (= 0 (secs-to-millis-long 0))) - (is (= 2 (secs-to-millis-long 0.002))) - (is (= 500 (secs-to-millis-long 0.5))) - (is (= 1000 (secs-to-millis-long 1))) - (is (= 1080 (secs-to-millis-long 1.08))) - (is (= 10000 (secs-to-millis-long 10))) - (is (= 10100 (secs-to-millis-long 10.1))) + (is (= 0 (Time/secsToMillisLong 0))) + (is (= 2 (Time/secsToMillisLong 0.002))) + (is (= 500 (Time/secsToMillisLong 0.5))) + (is (= 1000 (Time/secsToMillisLong 1))) + (is (= 1080 (Time/secsToMillisLong 1.08))) + (is (= 10000 (Time/secsToMillisLong 10))) + (is (= 10100 (Time/secsToMillisLong 10.1))) ) diff --git a/storm-core/test/clj/org/apache/storm/worker_test.clj b/storm-core/test/clj/org/apache/storm/worker_test.clj index 031b97e3365..6b6fede770e 100644 --- a/storm-core/test/clj/org/apache/storm/worker_test.clj +++ b/storm-core/test/clj/org/apache/storm/worker_test.clj @@ -16,7 +16,6 @@ (ns org.apache.storm.worker-test (:use [clojure test]) (:require [org.apache.storm.daemon [worker :as worker]]) - (:require [org.apache.storm [util :as util]]) (:require [conjure.core]) (:require [clj-time.core :as time]) (:require [clj-time.coerce :as coerce]) diff --git a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedConfigUtils.java b/storm-core/test/jvm/org/apache/storm/utils/staticmocking/ConfigUtilsInstaller.java similarity index 62% rename from storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedConfigUtils.java rename to storm-core/test/jvm/org/apache/storm/utils/staticmocking/ConfigUtilsInstaller.java index 6bd45d2a9fa..a6a31ca7b81 100644 --- a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedConfigUtils.java +++ b/storm-core/test/jvm/org/apache/storm/utils/staticmocking/ConfigUtilsInstaller.java @@ -14,18 +14,25 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.testing.staticmocking; +package org.apache.storm.utils.staticmocking; import org.apache.storm.utils.ConfigUtils; -public class MockedConfigUtils extends ConfigUtils implements AutoCloseable { +public class ConfigUtilsInstaller implements AutoCloseable { - public MockedConfigUtils() { - ConfigUtils.setInstance(this); + private ConfigUtils _oldInstance; + private ConfigUtils _curInstance; + + public ConfigUtilsInstaller(ConfigUtils instance) { + _oldInstance = ConfigUtils.setInstance(instance); + _curInstance = instance; } @Override public void close() throws Exception { - ConfigUtils.resetInstance(); + if (ConfigUtils.setInstance(_oldInstance) != _curInstance) { + throw new IllegalStateException( + "Instances of this resource must be closed in reverse order of opening."); + } } } \ No newline at end of file diff --git a/storm-core/test/jvm/org/apache/storm/utils/staticmocking/UtilsInstaller.java b/storm-core/test/jvm/org/apache/storm/utils/staticmocking/UtilsInstaller.java new file mode 100644 index 00000000000..106ec86896c --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/utils/staticmocking/UtilsInstaller.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version + * 2.0 (the "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.utils.staticmocking; + +import org.apache.storm.utils.Utils; + +public class UtilsInstaller implements AutoCloseable { + + private Utils _oldInstance; + private Utils _curInstance; + + public UtilsInstaller(Utils instance) { + _oldInstance = Utils.setInstance(instance); + _curInstance = instance; + } + + @Override + public void close() throws Exception { + if (Utils.setInstance(_oldInstance) != _curInstance) { + throw new IllegalStateException( + "Instances of this resource must be closed in reverse order of opening."); + } + } +} \ No newline at end of file diff --git a/storm-core/test/jvm/org/apache/storm/utils/staticmocking/package-info.java b/storm-core/test/jvm/org/apache/storm/utils/staticmocking/package-info.java new file mode 100644 index 00000000000..5825782348a --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/utils/staticmocking/package-info.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version + * 2.0 (the "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +/** + * Provides implementations for testing static methods. + * + * This package should not exist and is only necessary while we need to mock + * static methods. + * + * To mock static methods in java, we use a singleton. The class to mock must + * implement setInstance static method that accepts an instance of + * the selfsame class and returns the previous instance that was set. + * + * Example: + * + * + * public class MyClass { + * public static MyClass setInstance(MyClass c) { + * MyClass oldInstance = _instance; + * _instance = c; + * return oldInstance; + * } + * + * // Any method that we wish to mock must delegate to the singleton + * // instance's corresponding member method implementation + * public static int mockableFunction(String arg) { + * return _instance.mockableFunctionImpl(); + * } + * + * protected int mockableFunctionImpl(String arg) { + * return arg.size(); + * } + * } + * + * + * Each class that could be mocked should have an Installer class defined in + * this package that sets the instance on construction and implements the + * close method of {@link java.lang.AutoCloseable}. + * + * Example: + * + * + * class MyClassInstaller implementes AutoCloseable { + * private MyClass _oldInstance; + * private MyClass _curInstance; + * + * MyClassInstaller(MyClass instance) { + * _oldInstance = MyClass.setInstance(instance); + * _curInstance = instance; + * } + * + * @Override + * public void close() throws Exception { + * if (MyClass.setInstance(_oldInstance) != _curInstance) { + * throw new IllegalStateException( + * "Instances of this resource must be closed in reverse order of opening."); + * } + * } + * } + * + * + * To write a test with the mocked class instantiate a child class that + * implements the close method, and use try-with-resources. For example: + * + * + * MyClass mock = new MyClass() { + * protected int mockableFunctionImpl(String arg) { return 42; } + * }; + * + * try(mock) { + * AssertEqual(42, MyClass.mockableFunction("not 42 characters")); + * }; + * + * + * + * The resulting code remains thread-unsafe. + * + * This class should be removed when troublesome static methods have been + * replaced in the code. + */ +package org.apache.storm.testing.staticmocking;