Permalink
Browse files

merge master

  • Loading branch information...
nathanmarz committed Apr 29, 2012
2 parents b7d7962 + a942da6 commit d99f13d0ac93feeea1cb53c8eebc9a4009e51b75
View
@@ -1,5 +1,8 @@
## Unreleased
+NOTE: The change from 0.7.0 in which OutputCollector no longer assumes immutable inputs has been reverted to support optimized sending of tuples to colocated tasks
+
+ * Messages sent to colocated tasks are sent in-memory, skipping serialization (useful in conjunction with localOrShuffle grouping) (thanks xumingming)
* Upgrade to Clojure 1.4 (thanks sorenmacbeth)
* Can override the hostname that supervisors report using "storm.local.hostname" config.
* Make request timeout within DRPC server configurable via "drpc.request.timeout.secs"
@@ -10,6 +13,7 @@
* Added close method to OpaqueTransactionalSpout coordinator
* Added "storm dev-zookeeper" command for launching a local zookeeper server. Useful for testing a one node Storm cluster locally. Zookeeper dir configured with "dev.zookeeper.path"
* Use new style classes for Python multilang adapter (thanks hellp)
+ * Added "storm version" command
* Bug fix: Fixed criticial bug in opaque transactional topologies that would lead to duplicate messages when using pipelining
* Bug fix: Workers will now die properly if a ShellBolt subprocess dies (thanks tomo)
* Bug fix: Hide the BasicOutputCollector#getOutputter method, since it shouldn't be a publicly available method.
@@ -69,7 +73,7 @@
* Logging now always goes to logs/ in the Storm directory, regardless of where you launched the daemon (thanks haitaoyao)
* Improved Clojure DSL: can emit maps and Tuples implement the appropriate interfaces to integrate with Clojure's seq functions (thanks schleyfox)
* Added "ui.childopts" config (thanks ddillinger)
- * Bug fix: OutputCollector no longer assumes immutable inputs
+ * Bug fix: OutputCollector no longer assumes immutable inputs [NOTE: this was reverted in 0.7.2 because it conflicts with sending tuples to colocated tasks without serialization]
* Bug fix: DRPC topologies now throw a proper error when no DRPC servers are configured instead of NPE (thanks danharvey)
* Bug fix: Fix local mode so multiple topologies can be run on one LocalCluster
* Bug fix: "storm supervisor" now uses supervisor.childopts instead of nimbus.childopts (thanks ddillinger)
View
@@ -2,6 +2,8 @@ Storm is a distributed realtime computation system. Similar to how Hadoop provid
The [Rationale page](https://github.com/nathanmarz/storm/wiki/Rationale) on the wiki explains what Storm is and why it was built. The [video](http://www.infoq.com/presentations/Storm) and [slides](http://www.slideshare.net/nathanmarz/storm-distributed-and-faulttolerant-realtime-computation) of Storm's launch presentation are also good introductions to the project.
+Follow [@stormprocessor](https://twitter.com/stormprocessor) on Twitter for updates on the project.
+
## Documentation
Documentation and tutorials can be found on the [Storm wiki](http://github.com/nathanmarz/storm/wiki).
View
@@ -132,7 +132,7 @@ def activate(*args):
jvmtype="-client",
extrajars=[CONF_DIR, STORM_DIR + "/bin"])
-def list(*args):
+def listtopos(*args):
"""Syntax: [storm list]
List the running topologies and their statuses.
@@ -273,6 +273,17 @@ def dev_zookeeper():
jvmtype="-server",
extrajars=[STORM_DIR + "/log4j", STORM_DIR + "/conf"])
+def version():
+ """Syntax: [storm version]
+
+ Prints the version number of this Storm release.
+ """
+ releasefile = STORM_DIR + "/RELEASE"
+ if os.path.exists(releasefile):
+ print open(releasefile).readline().strip()
+ else:
+ print "Unknown"
+
def print_classpath():
"""Syntax: [storm classpath]
@@ -305,7 +316,7 @@ COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui
"drpc": drpc, "supervisor": supervisor, "localconfvalue": print_localconfvalue,
"remoteconfvalue": print_remoteconfvalue, "repl": repl, "classpath": print_classpath,
"activate": activate, "deactivate": deactivate, "rebalance": rebalance, "help": print_usage,
- "list": list, "dev-zookeeper": dev_zookeeper}
+ "list": listtopos, "dev-zookeeper": dev_zookeeper, "version": version}
def parse_config(config_list):
global CONFIG_OPTS
@@ -1,8 +1,9 @@
(ns backtype.storm.daemon.task
(:use [backtype.storm.daemon common])
(:use [backtype.storm bootstrap])
- (:import [java.util.concurrent ConcurrentLinkedQueue ConcurrentHashMap])
+ (:import [java.util.concurrent ConcurrentLinkedQueue ConcurrentHashMap LinkedBlockingQueue])
(:import [backtype.storm.hooks ITaskHook])
+ (:import [backtype.storm.tuple Tuple])
(:import [backtype.storm.hooks.info SpoutAckInfo SpoutFailInfo
EmitInfo BoltFailInfo BoltAckInfo])
(:require [backtype.storm [tuple :as tuple]]))
@@ -156,8 +157,10 @@
(.getThisTaskId topology-context)
stream))))
-(defn mk-task [conf storm-conf topology-context user-context storm-id mq-context cluster-state storm-active-atom transfer-fn suicide-fn]
+(defn mk-task [conf storm-conf topology-context user-context storm-id cluster-state storm-active-atom transfer-fn suicide-fn
+ receive-queue]
(let [task-id (.getThisTaskId topology-context)
+ worker-port (.getThisWorkerPort topology-context)
component-id (.getThisComponentId topology-context)
storm-conf (component-conf storm-conf topology-context component-id)
_ (log-message "Loading task " component-id ":" task-id)
@@ -196,9 +199,7 @@
stream->component->grouper (outbound-components topology-context user-context)
component->tasks (reverse-map task-info)
- ;; important it binds to virtual port before function returns
- puller (msg/bind mq-context storm-id task-id)
-
+
;; TODO: consider DRYing things up and moving stats
task-readable-name (get-readable-name topology-context)
@@ -239,7 +240,7 @@
_ (send-unanchored topology-context tasks-fn transfer-fn SYSTEM-STREAM-ID ["startup"])
executor-threads (dofor
[exec (with-error-reaction report-error-and-die
- (mk-executors task-object storm-conf puller tasks-fn
+ (mk-executors task-object storm-conf receive-queue tasks-fn
transfer-fn
storm-active-atom topology-context
user-context task-stats report-error))]
@@ -254,16 +255,16 @@
[this]
(log-message "Shutting down task " storm-id ":" task-id)
(reset! active false)
- ;; empty messages are skip messages (this unblocks the socket)
- (msg/send-local-task-empty mq-context storm-id task-id)
+ ;; put an empty message into receive-queue
+ ;; empty messages are skip messages (this unblocks the receive-queue.take thread)
+ (.put receive-queue (byte-array []))
(doseq [t all-threads]
(.interrupt t)
(.join t))
(doseq [hook (.getHooks user-context)]
(.cleanup hook))
(.remove-task-heartbeat! storm-cluster-state storm-id task-id)
(.disconnect storm-cluster-state)
- (.close puller)
(close-component task-object)
(log-message "Shut down task " storm-id ":" task-id))
DaemonCommon
@@ -290,7 +291,18 @@
(stats/spout-acked-tuple! task-stats (:stream tuple-info) time-delta)
))
-(defmethod mk-executors ISpout [^ISpout spout storm-conf puller tasks-fn transfer-fn storm-active-atom
+(defn mk-task-receiver [^LinkedBlockingQueue receive-queue ^KryoTupleDeserializer deserializer tuple-action-fn]
+ (fn []
+ (let [msg (.take receive-queue)
+ is-tuple? (instance? Tuple msg)]
+ (when (or is-tuple? (not (empty? msg))) ; skip empty messages (used during shutdown)
+ (log-debug "Processing message " msg)
+ (let [^Tuple tuple (if is-tuple? msg (.deserialize deserializer msg))]
+ (tuple-action-fn tuple)
+ ))
+ )))
+
+(defmethod mk-executors ISpout [^ISpout spout storm-conf ^LinkedBlockingQueue receive-queue tasks-fn transfer-fn storm-active-atom
^TopologyContext topology-context ^TopologyContext user-context
task-stats report-error-fn]
(let [wait-fn (fn [] @storm-active-atom)
@@ -349,10 +361,24 @@
)
(reportError [this error]
(report-error-fn error)
- ))]
+ ))
+ tuple-action-fn (fn [^Tuple tuple]
+ (let [id (.getValue tuple 0)
+ [spout-id tuple-finished-info start-time-ms] (.remove pending id)]
+ (when spout-id
+ (let [time-delta (time-delta-ms start-time-ms)]
+ (condp = (.getSourceStreamId tuple)
+ ACKER-ACK-STREAM-ID (.add event-queue #(ack-spout-msg spout user-context storm-conf spout-id
+ tuple-finished-info time-delta task-stats sampler))
+ ACKER-FAIL-STREAM-ID (.add event-queue #(fail-spout-msg spout user-context storm-conf spout-id
+ tuple-finished-info time-delta task-stats sampler))
+ )))
+ ;; TODO: on failure, emit tuple to failure stream
+ ))]
(log-message "Opening spout " component-id ":" task-id)
(.open spout storm-conf user-context (SpoutOutputCollector. output-collector))
(log-message "Opened spout " component-id ":" task-id)
+ ;; TODO: should redesign this to only use one thread
[(fn []
;; This design requires that spouts be non-blocking
(loop []
@@ -377,23 +403,7 @@
;; TODO: log that it's getting throttled
(Time/sleep 100)))
))
- (fn []
- (let [^bytes ser-msg (msg/recv puller)]
- ;; skip empty messages (used during shutdown)
- (when-not (empty? ser-msg)
- (let [tuple (.deserialize deserializer ser-msg)
- id (.getValue tuple 0)
- [spout-id tuple-finished-info start-time-ms] (.remove pending id)]
- (when spout-id
- (let [time-delta (time-delta-ms start-time-ms)]
- (condp = (.getSourceStreamId tuple)
- ACKER-ACK-STREAM-ID (.add event-queue #(ack-spout-msg spout user-context storm-conf spout-id
- tuple-finished-info time-delta task-stats sampler))
- ACKER-FAIL-STREAM-ID (.add event-queue #(fail-spout-msg spout user-context storm-conf spout-id
- tuple-finished-info time-delta task-stats sampler))
- ))))
- ;; TODO: on failure, emit tuple to failure stream
- )))
+ (mk-task-receiver receive-queue deserializer tuple-action-fn)
]
))
@@ -405,7 +415,7 @@
;; TODO: this portion is not thread safe (multiple threads updating same value at same time)
(.put pending key (bit-xor curr id))))
-(defmethod mk-executors IBolt [^IBolt bolt storm-conf puller tasks-fn transfer-fn storm-active-atom
+(defmethod mk-executors IBolt [^IBolt bolt storm-conf ^LinkedBlockingQueue receive-queue tasks-fn transfer-fn storm-active-atom
^TopologyContext topology-context ^TopologyContext user-context
task-stats report-error-fn]
(let [deserializer (KryoTupleDeserializer. storm-conf topology-context)
@@ -466,37 +476,34 @@
)))
(reportError [this error]
(report-error-fn error)
- ))]
+ ))
+ tuple-action-fn (fn [^Tuple tuple]
+ ;; synchronization needs to be done with a key provided by this bolt, otherwise:
+ ;; spout 1 sends synchronization (s1), dies, same spout restarts somewhere else, sends synchronization (s2) and incremental update. s2 and update finish before s1 -> lose the incremental update
+ ;; TODO: for state sync, need to first send sync messages in a loop and receive tuples until synchronization
+ ;; buffer other tuples until fully synchronized, then process all of those tuples
+ ;; then go into normal loop
+ ;; spill to disk?
+ ;; could be receiving incremental updates while waiting for sync or even a partial sync because of another failed task
+ ;; should remember sync requests and include a random sync id in the request. drop anything not related to active sync requests
+ ;; or just timeout the sync messages that are coming in until full sync is hit from that task
+ ;; need to drop incremental updates from tasks where waiting for sync. otherwise, buffer the incremental updates
+ ;; 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 " (.getThisTaskId topology-context))
+ (.put tuple-start-times tuple (System/currentTimeMillis))
+
+ (.execute bolt tuple))]
(log-message "Preparing bolt " component-id ":" task-id)
(.prepare bolt
storm-conf
user-context
(OutputCollector. output-collector))
(log-message "Prepared bolt " component-id ":" task-id)
;; TODO: can get any SubscribedState objects out of the context now
- [(fn []
- ;; synchronization needs to be done with a key provided by this bolt, otherwise:
- ;; spout 1 sends synchronization (s1), dies, same spout restarts somewhere else, sends synchronization (s2) and incremental update. s2 and update finish before s1 -> lose the incremental update
- ;; TODO: for state sync, need to first send sync messages in a loop and receive tuples until synchronization
- ;; buffer other tuples until fully synchronized, then process all of those tuples
- ;; then go into normal loop
- ;; spill to disk?
- ;; could be receiving incremental updates while waiting for sync or even a partial sync because of another failed task
- ;; should remember sync requests and include a random sync id in the request. drop anything not related to active sync requests
- ;; or just timeout the sync messages that are coming in until full sync is hit from that task
- ;; need to drop incremental updates from tasks where waiting for sync. otherwise, buffer the incremental updates
- (let [^bytes ser (msg/recv puller)]
- (when-not (empty? ser) ; skip empty messages (used during shutdown)
- (log-debug "Processing message")
- (let [tuple (.deserialize deserializer ser)]
- ;; 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 " (.getThisTaskId topology-context))
- (.put tuple-start-times tuple (System/currentTimeMillis))
-
- (.execute bolt tuple)
- ))))]
+ [(mk-task-receiver receive-queue deserializer tuple-action-fn)]
))
(defmethod close-component ISpout [spout]
Oops, something went wrong.

0 comments on commit d99f13d

Please sign in to comment.