From 98f4e619d54052b73a309d23ab7214953e4c7774 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 5 Feb 2015 10:08:05 -0800 Subject: [PATCH 01/89] STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. Signed-off-by: P. Taylor Goetz --- .../clj/backtype/storm/daemon/supervisor.clj | 158 ++++++++++-------- 1 file changed, 92 insertions(+), 66 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj index 820379e2715..582c3df69ad 100644 --- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj @@ -39,9 +39,9 @@ (defn- assignments-snapshot [storm-cluster-state callback assignment-versions] (let [storm-ids (.assignments storm-cluster-state callback)] - (let [new-assignments + (let [new-assignments (->> - (dofor [sid storm-ids] + (dofor [sid storm-ids] (let [recorded-version (:version (get assignment-versions sid))] (if-let [assignment-version (.assignment-version storm-cluster-state sid callback)] (if (= assignment-version recorded-version) @@ -50,10 +50,10 @@ {sid nil}))) (apply merge) (filter-val not-nil?))] - + {:assignments (into {} (for [[k v] new-assignments] [k (:data v)])) :versions new-assignments}))) - + (defn- read-my-executors [assignments-snapshot storm-id assignment-id] (let [assignment (get assignments-snapshot storm-id) my-executors (filter (fn [[_ [node _]]] (= node assignment-id)) @@ -144,7 +144,7 @@ ))) (defn- wait-for-worker-launch [conf id start-time] - (let [state (worker-state conf id)] + (let [state (worker-state conf id)] (loop [] (let [hb (.get state LS-WORKER-HEARTBEAT)] (when (and @@ -220,11 +220,14 @@ )) :assignment-versions (atom {}) :sync-retry (atom 0) + :download-lock (Object.) }) (defn sync-processes [supervisor] (let [conf (:conf supervisor) + download-lock (:download-lock supervisor) ^LocalState local-state (:local-state supervisor) + storm-cluster-state (:storm-cluster-state supervisor) assigned-executors (defaulted (.get local-state LS-LOCAL-ASSIGNMENTS) {}) now (current-time-secs) allocated (read-allocated-workers supervisor assigned-executors now) @@ -247,7 +250,7 @@ ;; 5. create local dir for worker id ;; 5. launch new workers (give worker-id, port, and supervisor-id) ;; 6. wait for workers launch - + (log-debug "Syncing processes") (log-debug "Assigned executors: " assigned-executors) (log-debug "Allocated: " allocated) @@ -268,25 +271,45 @@ (keys keepers)) (zipmap (vals new-worker-ids) (keys new-worker-ids)) )) + ;; check storm topology code dir exists before launching workers + (doseq [[port assignment] reassign-executors] + (let [downloaded-storm-ids (set (read-downloaded-storm-ids conf)) + storm-id (:storm-id assignment) + cached-assignment-info @ (:assignment-versions supervisor) + assignment-info (if (and (not-nil? cached-assignment-info) (contains? cached-assignment-info storm-id)) + (get cached-assignment-info storm-id) + (.assignment-info-with-version storm-cluster-state storm-id nil)) + storm-code-map (read-storm-code-locations assignment-info) + master-code-dir (if (contains? storm-code-map :data) (storm-code-map :data)) + stormroot (supervisor-stormdist-root conf storm-id)] + (if-not (or (contains? downloaded-storm-ids storm-id) (.exists (File. stormroot)) (nil? master-code-dir)) + (download-storm-code conf storm-id master-code-dir download-lock)))) + (wait-for-workers-launch conf (dofor [[port assignment] reassign-executors] - (let [id (new-worker-ids port)] - (log-message "Launching worker with assignment " - (pr-str assignment) - " for this supervisor " - (:supervisor-id supervisor) - " on port " - port - " with id " - id - ) - (launch-worker supervisor - (:storm-id assignment) - port - id) - id))) - )) + (let [id (new-worker-ids port)] + (try + (log-message "Launching worker with assignment " + (pr-str assignment) + " for this supervisor " + (:supervisor-id supervisor) + " on port " + port + " with id " + id) + (launch-worker supervisor + (:storm-id assignment) + port + id) + (catch java.io.FileNotFoundException e + (log-message "Unable to launch worker due to " + (.getMessage e))) + (catch java.io.IOException e + (log-message "Unable to launch worker due to " + (.getMessage e)))) + id)) + ))) (defn assigned-storm-ids-from-port-assignments [assignment] (->> assignment @@ -312,13 +335,14 @@ (defn mk-synchronize-supervisor [supervisor sync-processes event-manager processes-event-manager] (fn this [] (let [conf (:conf supervisor) + download-lock (:download-lock supervisor) storm-cluster-state (:storm-cluster-state supervisor) ^ISupervisor isupervisor (:isupervisor supervisor) ^LocalState local-state (:local-state supervisor) sync-callback (fn [& ignored] (.add event-manager this)) assignment-versions @(:assignment-versions supervisor) - {assignments-snapshot :assignments versions :versions} (assignments-snapshot - storm-cluster-state sync-callback + {assignments-snapshot :assignments versions :versions} (assignments-snapshot + storm-cluster-state sync-callback assignment-versions) storm-code-map (read-storm-code-locations assignments-snapshot) downloaded-storm-ids (set (read-downloaded-storm-ids conf)) @@ -336,7 +360,7 @@ (log-debug "Downloaded storm ids: " downloaded-storm-ids) (log-debug "All assignment: " all-assignment) (log-debug "New assignment: " new-assignment) - + ;; download code first ;; This might take awhile ;; - should this be done separately from usual monitoring? @@ -344,16 +368,7 @@ (doseq [[storm-id master-code-dir] storm-code-map] (when (and (not (downloaded-storm-ids storm-id)) (assigned-storm-ids storm-id)) - (log-message "Downloading code for storm id " - storm-id - " from " - master-code-dir) - (download-storm-code conf storm-id master-code-dir) - (log-message "Finished downloading code for storm id " - storm-id - " from " - master-code-dir) - )) + (download-storm-code conf storm-id master-code-dir download-lock))) (log-debug "Writing new assignment " (pr-str new-assignment)) @@ -389,7 +404,7 @@ (.prepare isupervisor conf (supervisor-isupervisor-dir conf)) (FileUtils/cleanDirectory (File. (supervisor-tmp-dir conf))) (let [supervisor (supervisor-data conf shared-context isupervisor) - [event-manager processes-event-manager :as managers] [(event/event-manager false) (event/event-manager false)] + [event-manager processes-event-manager :as managers] [(event/event-manager false) (event/event-manager false)] sync-processes (partial sync-processes supervisor) synchronize-supervisor (mk-synchronize-supervisor supervisor sync-processes event-manager processes-event-manager) heartbeat-fn (fn [] (.supervisor-heartbeat! @@ -452,27 +467,37 @@ ;; distributed implementation (defmethod download-storm-code - :distributed [conf storm-id master-code-dir] - ;; Downloading to permanent location is atomic - (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid)) - stormroot (supervisor-stormdist-root conf storm-id)] + :distributed [conf storm-id master-code-dir download-lock] + ;; Downloading to permanent location is atomic + (let [tmproot (str (supervisor-tmp-dir conf) file-path-separator (uuid)) + stormroot (supervisor-stormdist-root conf storm-id)] + (locking download-lock + (log-message "Downloading code for storm id " + storm-id + " from " + master-code-dir) + (FileUtils/forceMkdir (File. tmproot)) - + (Utils/downloadFromMaster conf (master-stormjar-path master-code-dir) (supervisor-stormjar-path tmproot)) (Utils/downloadFromMaster conf (master-stormcode-path master-code-dir) (supervisor-stormcode-path tmproot)) (Utils/downloadFromMaster conf (master-stormconf-path master-code-dir) (supervisor-stormconf-path tmproot)) (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot) (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) - )) + (log-message "Finished downloading code for storm id " + storm-id + " from " + master-code-dir)) + )) (defn jlp [stormroot conf] (let [resource-root (str stormroot File/separator RESOURCES-SUBDIR) os (clojure.string/replace (System/getProperty "os.name") #"\s+" "_") arch (System/getProperty "os.arch") arch-resource-root (str resource-root File/separator os "-" arch)] - (str arch-resource-root File/pathSeparator resource-root File/pathSeparator (conf JAVA-LIBRARY-PATH)))) + (str arch-resource-root File/pathSeparator resource-root File/pathSeparator (conf JAVA-LIBRARY-PATH)))) -(defn substitute-childopts +(defn substitute-childopts "Generates runtime childopts by replacing keys with topology-id, worker-id, port" [value worker-id topology-id port] (let [replacement-map {"%ID%" (str port) @@ -481,7 +506,7 @@ "%WORKER-PORT%" (str port)} sub-fn #(reduce (fn [string entry] (apply clojure.string/replace string entry)) - % + % replacement-map)] (cond (nil? value) nil @@ -558,27 +583,28 @@ first )) (defmethod download-storm-code - :local [conf storm-id master-code-dir] - (let [stormroot (supervisor-stormdist-root conf storm-id)] - (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot)) - (let [classloader (.getContextClassLoader (Thread/currentThread)) - resources-jar (resources-jar) - url (.getResource classloader RESOURCES-SUBDIR) - target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)] - (cond - resources-jar - (do - (log-message "Extracting resources from jar at " resources-jar " to " target-dir) - (extract-dir-from-jar resources-jar RESOURCES-SUBDIR stormroot)) - url - (do - (log-message "Copying resources at " (URI. (str url)) " to " target-dir) - (if (= (.getProtocol url) "jar" ) - (extract-dir-from-jar (.getFile (.getJarFileURL (.openConnection url))) RESOURCES-SUBDIR stormroot) - (FileUtils/copyDirectory (File. (.getPath (URI. (str url)))) (File. target-dir))) - ) - ) - ))) + :local [conf storm-id master-code-dir download-lock] + (let [stormroot (supervisor-stormdist-root conf storm-id)] + (locking download-lock + (FileUtils/copyDirectory (File. master-code-dir) (File. stormroot)) + (let [classloader (.getContextClassLoader (Thread/currentThread)) + resources-jar (resources-jar) + url (.getResource classloader RESOURCES-SUBDIR) + target-dir (str stormroot file-path-separator RESOURCES-SUBDIR)] + (cond + resources-jar + (do + (log-message "Extracting resources from jar at " resources-jar " to " target-dir) + (extract-dir-from-jar resources-jar RESOURCES-SUBDIR stormroot)) + url + (do + (log-message "Copying resources at " (URI. (str url)) " to " target-dir) + (if (= (.getProtocol url) "jar" ) + (extract-dir-from-jar (.getFile (.getJarFileURL (.openConnection url))) RESOURCES-SUBDIR stormroot) + (FileUtils/copyDirectory (File. (.getPath (URI. (str url)))) (File. target-dir))) + ) + ) + )))) (defmethod launch-worker :local [supervisor storm-id port worker-id] From a1e5893e1b94c224d39fedf11583b216c21351c8 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 24 Feb 2015 15:46:12 -0500 Subject: [PATCH 02/89] update changelog for STORM-130 --- CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index b7a7f412a22..5839b7a27af 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,6 @@ +## 0.9.4 + * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. + ## 0.9.3-rc2 * STORM-558: change "swap!" to "reset!" to fix assignment-versions in supervisor * STORM-555: Storm json response should set charset to UTF-8 From 62788f295bb1fb1cc83b99c30f82beb40eea5f25 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 24 Feb 2015 18:03:40 -0500 Subject: [PATCH 03/89] port STORM-329 fix to 0.9.x --- conf/defaults.yaml | 2 +- .../src/clj/backtype/storm/daemon/worker.clj | 45 +- .../clj/backtype/storm/messaging/local.clj | 2 +- .../storm/messaging/ConnectionWithStatus.java | 32 + .../storm/messaging/netty/Client.java | 642 +++++++++++++----- .../storm/messaging/netty/Server.java | 122 +++- .../netty/StormClientPipelineFactory.java | 2 +- .../storm/messaging/netty_unit_test.clj | 70 +- .../test/clj/backtype/storm/worker_test.clj | 38 ++ 9 files changed, 725 insertions(+), 230 deletions(-) create mode 100644 storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java create mode 100644 storm-core/test/clj/backtype/storm/worker_test.clj diff --git a/conf/defaults.yaml b/conf/defaults.yaml index 0050227a343..e2b33000aab 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -109,7 +109,7 @@ zmq.hwm: 0 storm.messaging.netty.server_worker_threads: 1 storm.messaging.netty.client_worker_threads: 1 storm.messaging.netty.buffer_size: 5242880 #5MB buffer -# Since nimbus.task.launch.secs and supervisor.worker.start.timeout.secs are 120, other workers should also wait at least that long before giving up on connecting to the other worker. +# Since nimbus.task.launch.secs and supervisor.worker.start.timeout.secs are 120, other workers should also wait at least that long before giving up on connecting to the other worker. The reconnection period need also be bigger than storm.zookeeper.session.timeout(default is 20s), so that we can abort the reconnection when the target worker is dead. storm.messaging.netty.max_retries: 300 storm.messaging.netty.max_wait_ms: 1000 storm.messaging.netty.min_wait_ms: 100 diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj index f1c4d3a272a..d4c946745a9 100644 --- a/storm-core/src/clj/backtype/storm/daemon/worker.clj +++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj @@ -21,7 +21,7 @@ (:import [java.util ArrayList HashMap]) (:import [backtype.storm.utils TransferDrainer]) (:import [backtype.storm.messaging TransportFactory]) - (:import [backtype.storm.messaging TaskMessage IContext IConnection]) + (:import [backtype.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status]) (:gen-class)) (bootstrap) @@ -201,6 +201,10 @@ :worker-id worker-id :cluster-state cluster-state :storm-cluster-state storm-cluster-state + ;; when worker bootup, worker will start to setup initial connections to + ;; other workers. When all connection is ready, we will enable this flag + ;; and spout and bolt will be activated. + :worker-active-flag (atom false) :storm-active-atom (atom false) :executors executors :task-ids (->> receive-queue-map keys (map int) sort) @@ -308,7 +312,7 @@ (let [base (.storm-base (:storm-cluster-state worker) (:storm-id worker) callback)] (reset! (:storm-active-atom worker) - (= :active (-> base :status :type)) + (and (= :active (-> base :status :type)) @(:worker-active-flag worker)) )) )) @@ -330,6 +334,37 @@ (.send drainer node+port->socket))) (.clear drainer)))))) +;; Check whether this messaging connection is ready to send data +(defn is-connection-ready [^IConnection connection] + (if (instance? ConnectionWithStatus connection) + (let [^ConnectionWithStatus connection connection + status (.status connection)] + (= status ConnectionWithStatus$Status/Ready)) + true)) + +;; all connections are ready +(defn all-connections-ready [worker] + (let [connections (vals @(:cached-node+port->socket worker))] + (every? is-connection-ready connections))) + +;; we will wait all connections to be ready and then activate the spout/bolt +;; when the worker bootup +(defn activate-worker-when-all-connections-ready + [worker] + (let [timer (:refresh-active-timer worker) + delay-secs 0 + recur-secs 1] + (schedule timer + delay-secs + (fn this [] + (if (all-connections-ready worker) + (do + (log-message "All connections are ready for worker " (:assignment-id worker) ":" (:port worker) + " with id "(:worker-id worker)) + (reset! (:worker-active-flag worker) true)) + (schedule timer recur-secs this :check-active false) + ))))) + (defn launch-receive-thread [worker] (log-message "Launching receive-thread for " (:assignment-id worker) ":" (:port worker)) (msg-loader/launch-receive-thread! @@ -373,14 +408,18 @@ _ (schedule-recurring (:heartbeat-timer worker) 0 (conf WORKER-HEARTBEAT-FREQUENCY-SECS) heartbeat-fn) _ (schedule-recurring (:executor-heartbeat-timer worker) 0 (conf TASK-HEARTBEAT-FREQUENCY-SECS) #(do-executor-heartbeats worker :executors @executors)) + receive-thread-shutdown (launch-receive-thread worker) refresh-connections (mk-refresh-connections worker) _ (refresh-connections nil) + + _ (activate-worker-when-all-connections-ready worker) + _ (refresh-storm-active worker nil) + _ (reset! executors (dofor [e (:executors worker)] (executor/mk-executor worker e))) - receive-thread-shutdown (launch-receive-thread worker) transfer-tuples (mk-transfer-tuples-handler worker) diff --git a/storm-core/src/clj/backtype/storm/messaging/local.clj b/storm-core/src/clj/backtype/storm/messaging/local.clj index 801f22d3417..4aa67abec5b 100644 --- a/storm-core/src/clj/backtype/storm/messaging/local.clj +++ b/storm-core/src/clj/backtype/storm/messaging/local.clj @@ -70,4 +70,4 @@ (defn mk-context [] (let [context (LocalContext. nil nil)] (.prepare ^IContext context nil) - context)) \ No newline at end of file + context)) diff --git a/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java b/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java new file mode 100644 index 00000000000..38abc19edfc --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java @@ -0,0 +1,32 @@ +package backtype.storm.messaging; + +public abstract class ConnectionWithStatus implements IConnection { + + public static enum Status { + + /** + * we are establishing a active connection with target host. The new data + * sending request can be buffered for future sending, or dropped(cases like + * there is no enough memory). It varies with difference IConnection + * implementations. + */ + Connecting, + + /** + * We have a alive connection channel, which can be used to transfer data. + */ + Ready, + + /** + * The connection channel is closed or being closed. We don't accept further + * data sending or receiving. All data sending request will be dropped. + */ + Closed + }; + + /** + * whether this connection is available to transfer data + */ + public abstract Status status(); + +} \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index c516b637ef7..ae8a8d52d55 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -30,317 +30,601 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; +import java.net.SocketAddress; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Random; +import java.util.HashMap; +import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -public class Client implements IConnection { +import com.google.common.util.concurrent.*; +import org.jboss.netty.bootstrap.ClientBootstrap; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.ConnectionWithStatus; +import backtype.storm.metric.api.IStatefulObject; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.utils.StormBoundedExponentialBackoffRetry; +import backtype.storm.utils.Utils; + +/** + * A Netty client for sending task messages to a remote destination (Netty server). + * + * Implementation details: + * + * - Sending messages, i.e. writing to the channel, is performed asynchronously. + * - Messages are sent in batches to optimize for network throughput at the expense of network latency. The message + * batch size is configurable. + * - Connecting and reconnecting are performed asynchronously. + * - Note: The current implementation drops any messages that are being enqueued for sending if the connection to + * the remote destination is currently unavailable. + * - A background flusher thread is run in the background. It will, at fixed intervals, check for any pending messages + * (i.e. messages buffered in memory) and flush them to the remote destination iff background flushing is currently + * enabled. + */ +public class Client extends ConnectionWithStatus implements IStatefulObject { + private static final Logger LOG = LoggerFactory.getLogger(Client.class); private static final String PREFIX = "Netty-Client-"; - private final int max_retries; - private final int base_sleep_ms; - private final int max_sleep_ms; + private static final long NO_DELAY_MS = 0L; + private static final long MINIMUM_INITIAL_DELAY_MS = 30000L; + private static final long PENDING_MESSAGES_FLUSH_TIMEOUT_MS = 600000L; + private static final long PENDING_MESSAGES_FLUSH_INTERVAL_MS = 1000L; + private static final long DISTANT_FUTURE_TIME_MS = Long.MAX_VALUE; + private final StormBoundedExponentialBackoffRetry retryPolicy; - private AtomicReference channelRef; private final ClientBootstrap bootstrap; - private InetSocketAddress remote_addr; + private final InetSocketAddress dstAddress; + protected final String dstAddressPrefixedName; + + /** + * The channel used for all write operations from this client to the remote destination. + */ + private final AtomicReference channelRef = new AtomicReference(null); + + + /** + * Maximum number of reconnection attempts we will perform after a disconnect before giving up. + */ + private final int maxReconnectionAttempts; + + /** + * Total number of connection attempts. + */ + private final AtomicInteger totalConnectionAttempts = new AtomicInteger(0); + + /** + * Number of connection attempts since the last disconnect. + */ + private final AtomicInteger connectionAttempts = new AtomicInteger(0); + + /** + * Number of messages successfully sent to the remote destination. + */ + private final AtomicInteger messagesSent = new AtomicInteger(0); + + /** + * Number of messages that could not be sent to the remote destination. + */ + private final AtomicInteger messagesLost = new AtomicInteger(0); + + /** + * Number of messages buffered in memory. + */ + private final AtomicLong pendingMessages = new AtomicLong(0); + + /** + * This flag is set to true if and only if a client instance is being closed. + */ + private volatile boolean closing = false; + + /** + * When set to true, then the background flusher thread will flush any pending messages on its next run. + */ + private final AtomicBoolean backgroundFlushingEnabled = new AtomicBoolean(false); - private final Random random = new Random(); - private final ChannelFactory factory; - private final int buffer_size; - private boolean closing; + /** + * The absolute time (in ms) when the next background flush should be performed. + * + * Note: The flush operation will only be performed if backgroundFlushingEnabled is true, too. + */ + private final AtomicLong nextBackgroundFlushTimeMs = new AtomicLong(DISTANT_FUTURE_TIME_MS); - private int messageBatchSize; + /** + * The time interval (in ms) at which the background flusher thread will be run to check for any pending messages + * to be flushed. + */ + private final int flushCheckIntervalMs; - private AtomicLong pendings; + /** + * How many messages should be batched together before sending them to the remote destination. + * + * Messages are batched to optimize network throughput at the expense of latency. + */ + private final int messageBatchSize; - MessageBatch messageBatch = null; - private AtomicLong flushCheckTimer; - private int flushCheckInterval; - private ScheduledExecutorService scheduler; + private MessageBatch messageBatch = null; + private final ListeningScheduledExecutorService scheduler; + protected final Map stormConf; @SuppressWarnings("rawtypes") - Client(Map storm_conf, ChannelFactory factory, - ScheduledExecutorService scheduler, String host, int port) { - this.factory = factory; - this.scheduler = scheduler; - channelRef = new AtomicReference(null); + Client(Map stormConf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { closing = false; - pendings = new AtomicLong(0); - flushCheckTimer = new AtomicLong(Long.MAX_VALUE); - - // Configure - buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); - max_retries = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES)); - base_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); - max_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); - retryPolicy = new StormBoundedExponentialBackoffRetry(base_sleep_ms, max_sleep_ms, max_retries); - - this.messageBatchSize = Utils.getInt(storm_conf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); + this.stormConf = stormConf; + this.scheduler = MoreExecutors.listeningDecorator(scheduler); + int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); + LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize); + messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); + flushCheckIntervalMs = Utils.getInt(stormConf.get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); + + maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES)); + int minWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); + int maxWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); + retryPolicy = new StormBoundedExponentialBackoffRetry(minWaitMs, maxWaitMs, maxReconnectionAttempts); + + // Initiate connection to remote destination + bootstrap = createClientBootstrap(factory, bufferSize); + dstAddress = new InetSocketAddress(host, port); + dstAddressPrefixedName = prefixedName(dstAddress); + connect(NO_DELAY_MS); - flushCheckInterval = Utils.getInt(storm_conf.get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); // default 10 ms - - LOG.info("New Netty Client, connect to " + host + ", " + port - + ", config: " + ", buffer_size: " + buffer_size); + // Launch background flushing thread + pauseBackgroundFlushing(); + long initialDelayMs = Math.min(MINIMUM_INITIAL_DELAY_MS, maxWaitMs * maxReconnectionAttempts); + scheduler.scheduleWithFixedDelay(createBackgroundFlusher(), initialDelayMs, flushCheckIntervalMs, + TimeUnit.MILLISECONDS); + } - bootstrap = new ClientBootstrap(factory); + private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { + ClientBootstrap bootstrap = new ClientBootstrap(factory); bootstrap.setOption("tcpNoDelay", true); - bootstrap.setOption("sendBufferSize", buffer_size); + bootstrap.setOption("sendBufferSize", bufferSize); bootstrap.setOption("keepAlive", true); - - // Set up the pipeline factory. bootstrap.setPipelineFactory(new StormClientPipelineFactory(this)); + return bootstrap; + } - // Start the connection attempt. - remote_addr = new InetSocketAddress(host, port); + private String prefixedName(InetSocketAddress dstAddress) { + if (null != dstAddress) { + return PREFIX + dstAddress.toString(); + } + return ""; + } - // setup the connection asyncly now - scheduler.execute(new Runnable() { + private Runnable createBackgroundFlusher() { + return new Runnable() { @Override public void run() { - connect(); + if(!closing && backgroundFlushingEnabled.get() && nowMillis() > nextBackgroundFlushTimeMs.get()) { + LOG.debug("flushing {} pending messages to {} in background", messageBatch.size(), + dstAddressPrefixedName); + flushPendingMessages(); + } + } + }; + } + + private void pauseBackgroundFlushing() { + backgroundFlushingEnabled.set(false); } - }); - Runnable flusher = new Runnable() { - @Override - public void run() { + private void resumeBackgroundFlushing() { + backgroundFlushingEnabled.set(true); + } - if(!closing) { - long flushCheckTime = flushCheckTimer.get(); - long now = System.currentTimeMillis(); - if (now > flushCheckTime) { + private synchronized void flushPendingMessages() { Channel channel = channelRef.get(); - if (null != channel && channel.isWritable()) { - flush(channel); + if (containsMessages(messageBatch)) { + if (connectionEstablished(channel)) { + if (channel.isWritable()) { + pauseBackgroundFlushing(); + MessageBatch toBeFlushed = messageBatch; + flushMessages(channel, toBeFlushed); + messageBatch = null; + } + else if (closing) { + // Ensure background flushing is enabled so that we definitely have a chance to re-try the flush + // operation in case the client is being gracefully closed (where we have a brief time window where + // the client will wait for pending messages to be sent). + resumeBackgroundFlushing(); + } } + else { + closeChannelAndReconnect(channel); } } - } - }; - long initialDelay = Math.min(30L * 1000, max_sleep_ms * max_retries); //max wait for 30s - scheduler.scheduleWithFixedDelay(flusher, initialDelay, flushCheckInterval, TimeUnit.MILLISECONDS); + private long nowMillis() { + return System.currentTimeMillis(); } /** * We will retry connection with exponential back-off policy */ - private synchronized void connect() { + private synchronized void connect(long delayMs) { try { + if (closing) { + return; + } - Channel channel = channelRef.get(); - if (channel != null && channel.isConnected()) { + if (connectionEstablished(channelRef.get())) { return; } - int tried = 0; - //setting channel to null to make sure we throw an exception when reconnection fails - channel = null; - while (tried <= max_retries) { + connectionAttempts.getAndIncrement(); + if (reconnectingAllowed()) { + totalConnectionAttempts.getAndIncrement(); + LOG.info("connection attempt {} to {} scheduled to run in {} ms", connectionAttempts.get(), + dstAddressPrefixedName, delayMs); + ListenableFuture channelFuture = scheduler.schedule( + new Connector(dstAddress, connectionAttempts.get()), delayMs, TimeUnit.MILLISECONDS); + Futures.addCallback(channelFuture, new FutureCallback() { + @Override public void onSuccess(Channel result) { + if (connectionEstablished(result)) { + setChannel(result); + LOG.info("connection established to {}", dstAddressPrefixedName); + connectionAttempts.set(0); + } + else { + reconnectAgain(new RuntimeException("Returned channel was actually not established")); + } + } - LOG.info("Reconnect started for {}... [{}]", name(), tried); - LOG.debug("connection started..."); + @Override public void onFailure(Throwable t) { + reconnectAgain(t); + } - ChannelFuture future = bootstrap.connect(remote_addr); - future.awaitUninterruptibly(); - Channel current = future.getChannel(); - if (!future.isSuccess()) { - if (null != current) { - current.close(); + private void reconnectAgain(Throwable t) { + String baseMsg = String.format("connection attempt %s to %s failed", connectionAttempts, + dstAddressPrefixedName); + String failureMsg = (t == null)? baseMsg : baseMsg + ": " + t.toString(); + LOG.error(failureMsg); + long nextDelayMs = retryPolicy.getSleepTimeMs(connectionAttempts.get(), 0); + connect(nextDelayMs); } - } else { - channel = current; - break; + }); + } + else { + close(); + throw new RuntimeException("Giving up to connect to " + dstAddressPrefixedName + " after " + + connectionAttempts + " failed attempts"); + } + } + catch (Exception e) { + throw new RuntimeException("Failed to connect to " + dstAddressPrefixedName, e); } - Thread.sleep(retryPolicy.getSleepTimeMs(tried, 0)); - tried++; } - if (null != channel) { - LOG.info("connection established to a remote host " + name() + ", " + channel.toString()); + + private void setChannel(Channel channel) { channelRef.set(channel); - } else { - close(); - throw new RuntimeException("Remote address is not reachable. We will close this client " + name()); + } + + private boolean reconnectingAllowed() { + return !closing && connectionAttempts.get() <= (maxReconnectionAttempts + 1); + } + + private boolean connectionEstablished(Channel channel) { + // Because we are using TCP (which is a connection-oriented transport unlike UDP), a connection is only fully + // established iff the channel is connected. That is, a TCP-based channel must be in the CONNECTED state before + // anything can be read or written to the channel. + // + // See: + // - http://netty.io/3.9/api/org/jboss/netty/channel/ChannelEvent.html + // - http://stackoverflow.com/questions/13356622/what-are-the-netty-channel-state-transitions + return channel != null && channel.isConnected(); + } + + /** + * Note: Storm will check via this method whether a worker can be activated safely during the initial startup of a + * topology. The worker will only be activated once all of the its connections are ready. + */ + @Override + public Status status() { + if (closing) { + return Status.Closed; + } + else if (!connectionEstablished(channelRef.get())) { + return Status.Connecting; } - } catch (InterruptedException e) { - throw new RuntimeException("connection failed " + name(), e); + else { + return Status.Ready; } } /** - * Enqueue task messages to be sent to server + * Receiving messages is not supported by a client. + * + * @throws java.lang.UnsupportedOperationException whenever this method is being called. */ - synchronized public void send(Iterator msgs) { + @Override + public Iterator recv(int flags, int clientId) { + throw new UnsupportedOperationException("Client connection should not receive any messages"); + } + + @Override + public void send(int taskId, byte[] payload) { + TaskMessage msg = new TaskMessage(taskId, payload); + List wrapper = new ArrayList(1); + wrapper.add(msg); + send(wrapper.iterator()); + } - // throw exception if the client is being closed + /** + * Enqueue task messages to be sent to the remote destination (cf. `host` and `port`). + */ + @Override + public synchronized void send(Iterator msgs) { if (closing) { - throw new RuntimeException("Client is being closed, and does not take requests any more"); + int numMessages = iteratorSize(msgs); + LOG.warn("discarding {} messages because the Netty client to {} is being closed", numMessages, + dstAddressPrefixedName); + return; } - if (null == msgs || !msgs.hasNext()) { + if (!hasMessages(msgs)) { return; } Channel channel = channelRef.get(); - if (null == channel) { - connect(); - channel = channelRef.get(); + if (!connectionEstablished(channel)) { + // Closing the channel and reconnecting should be done before handling the messages. + closeChannelAndReconnect(channel); + handleMessagesWhenConnectionIsUnavailable(msgs); + return; } + // Collect messages into batches (to optimize network throughput), then flush them. while (msgs.hasNext()) { - if (!channel.isConnected()) { - connect(); - channel = channelRef.get(); - } TaskMessage message = msgs.next(); - if (null == messageBatch) { + if (messageBatch == null) { messageBatch = new MessageBatch(messageBatchSize); } messageBatch.add(message); if (messageBatch.isFull()) { MessageBatch toBeFlushed = messageBatch; - flushRequest(channel, toBeFlushed); + flushMessages(channel, toBeFlushed); messageBatch = null; } } - if (null != messageBatch && !messageBatch.isEmpty()) { - if (channel.isWritable()) { - flushCheckTimer.set(Long.MAX_VALUE); - - // Flush as fast as we can to reduce the latency + // Handle any remaining messages in case the "last" batch was not full. + if (containsMessages(messageBatch)) { + if (connectionEstablished(channel) && channel.isWritable()) { + // We can write to the channel, so we flush the remaining messages immediately to minimize latency. + pauseBackgroundFlushing(); MessageBatch toBeFlushed = messageBatch; messageBatch = null; - flushRequest(channel, toBeFlushed); + flushMessages(channel, toBeFlushed); + } + else { + // We cannot write to the channel, which means Netty's internal write buffer is full. + // In this case, we buffer the remaining messages and wait for the next messages to arrive. + // + // Background: + // Netty 3.x maintains an internal write buffer with a high water mark for each channel (default: 64K). + // This represents the amount of data waiting to be flushed to operating system buffers. If the + // outstanding data exceeds this value then the channel is set to non-writable. When this happens, a + // INTEREST_CHANGED channel event is triggered. Netty sets the channel to writable again once the data + // has been flushed to the system buffers. + // + // See http://stackoverflow.com/questions/14049260 + resumeBackgroundFlushing(); + nextBackgroundFlushTimeMs.set(nowMillis() + flushCheckIntervalMs); + } + } - } else { - // when channel is NOT writable, it means the internal netty buffer is full. - // In this case, we can try to buffer up more incoming messages. - flushCheckTimer.set(System.currentTimeMillis() + flushCheckInterval); } + + private boolean hasMessages(Iterator msgs) { + return msgs != null && msgs.hasNext(); } + /** + * We will drop pending messages and let at-least-once message replay kick in. + * + * Another option would be to buffer the messages in memory. But this option has the risk of causing OOM errors, + * especially for topologies that disable message acking because we don't know whether the connection recovery will + * succeed or not, and how long the recovery will take. + */ + private void handleMessagesWhenConnectionIsUnavailable(Iterator msgs) { + LOG.error("connection to {} is unavailable", dstAddressPrefixedName); + dropPendingMessages(msgs); } - public String name() { - if (null != remote_addr) { - return PREFIX + remote_addr.toString(); - } - return ""; + private void dropPendingMessages(Iterator msgs) { + // We consume the iterator by traversing and thus "emptying" it. + int msgCount = iteratorSize(msgs); + LOG.error("dropping {} pending message(s) destined for {}", msgCount, dstAddressPrefixedName); } - private synchronized void flush(Channel channel) { - if (!closing) { - if (null != messageBatch && !messageBatch.isEmpty()) { - MessageBatch toBeFlushed = messageBatch; - flushCheckTimer.set(Long.MAX_VALUE); - flushRequest(channel, toBeFlushed); - messageBatch = null; + private int iteratorSize(Iterator msgs) { + int size = 0; + if (msgs != null) { + while (msgs.hasNext()) { + size++; + msgs.next(); } } + return size; } /** - * gracefully close this client. + * Asynchronously writes the message batch to the channel. * - * We will send all existing requests, and then invoke close_n_release() - * method + * If the write operation fails, then we will close the channel and trigger a reconnect. */ - public synchronized void close() { - if (!closing) { - closing = true; - LOG.info("Closing Netty Client " + name()); + private synchronized void flushMessages(Channel channel, final MessageBatch batch) { + if (!containsMessages(batch)) { + return; + } + + final int numMessages = batch.size(); + pendingMessages.getAndAdd(numMessages); + LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString()); + ChannelFuture future = channel.write(batch); + future.addListener(new ChannelFutureListener() { + + public void operationComplete(ChannelFuture future) throws Exception { + pendingMessages.getAndAdd(0 - numMessages); + if (future.isSuccess()) { + LOG.debug("sent {} messages to {}", numMessages, dstAddressPrefixedName); + messagesSent.getAndAdd(batch.size()); + } + else { + LOG.warn("failed to send {} messages to {}: {}", numMessages, dstAddressPrefixedName, + future.getCause()); + closeChannelAndReconnect(future.getChannel()); + messagesLost.getAndAdd(numMessages); + } + } + + }); + } - if (null != messageBatch && !messageBatch.isEmpty()) { - MessageBatch toBeFlushed = messageBatch; - Channel channel = channelRef.get(); + private synchronized void closeChannelAndReconnect(Channel channel) { if (channel != null) { - flushRequest(channel, toBeFlushed); + channel.close(); + if (channelRef.compareAndSet(channel, null)) { + connect(NO_DELAY_MS); + } } - messageBatch = null; } - //wait for pendings to exit - final long timeoutMilliSeconds = 600 * 1000; //600 seconds - final long start = System.currentTimeMillis(); + private boolean containsMessages(MessageBatch batch) { + return batch != null && !batch.isEmpty(); + } - LOG.info("Waiting for pending batchs to be sent with "+ name() + "..., timeout: {}ms, pendings: {}", timeoutMilliSeconds, pendings.get()); + /** + * Gracefully close this client. + * + * We will attempt to send any pending messages (i.e. messages currently buffered in memory) before closing the + * client. + */ + @Override + public void close() { + if (!closing) { + LOG.info("closing Netty Client {}", dstAddressPrefixedName); + // Set closing to true to prevent any further reconnection attempts. + closing = true; + flushPendingMessages(); + waitForPendingMessagesToBeSent(); + closeChannel(); + } + } - while(pendings.get() != 0) { + private synchronized void waitForPendingMessagesToBeSent() { + LOG.info("waiting up to {} ms to send {} pending messages to {}", + PENDING_MESSAGES_FLUSH_TIMEOUT_MS, pendingMessages.get(), dstAddressPrefixedName); + long totalPendingMsgs = pendingMessages.get(); + long startMs = nowMillis(); + while (pendingMessages.get() != 0) { try { - long delta = System.currentTimeMillis() - start; - if (delta > timeoutMilliSeconds) { - LOG.error("Timeout when sending pending batchs with {}..., there are still {} pending batchs not sent", name(), pendings.get()); + long deltaMs = nowMillis() - startMs; + if (deltaMs > PENDING_MESSAGES_FLUSH_TIMEOUT_MS) { + LOG.error("failed to send all pending messages to {} within timeout, {} of {} messages were not " + + "sent", dstAddressPrefixedName, pendingMessages.get(), totalPendingMsgs); break; + } + Thread.sleep(PENDING_MESSAGES_FLUSH_INTERVAL_MS); } - Thread.sleep(1000); //sleep 1s - } catch (InterruptedException e) { + catch (InterruptedException e) { break; } } - close_n_release(); - } } - /** - * close_n_release() is invoked after all messages have been sent. - */ - private void close_n_release() { + private synchronized void closeChannel() { if (channelRef.get() != null) { channelRef.get().close(); - LOG.debug("channel {} closed",remote_addr); + LOG.debug("channel to {} closed", dstAddressPrefixedName); } } @Override - public Iterator recv(int flags, int clientId) { - throw new RuntimeException("Client connection should not receive any messages"); + public Object getState() { + LOG.info("Getting metrics for client connection to {}", dstAddressPrefixedName); + HashMap ret = new HashMap(); + ret.put("reconnects", totalConnectionAttempts.getAndSet(0)); + ret.put("sent", messagesSent.getAndSet(0)); + ret.put("pending", pendingMessages.get()); + ret.put("lostOnSend", messagesLost.getAndSet(0)); + ret.put("dest", dstAddress.toString()); + String src = srcAddressName(); + if (src != null) { + ret.put("src", src); + } + return ret; } - @Override - public void send(int taskId, byte[] payload) { - TaskMessage msg = new TaskMessage(taskId, payload); - List wrapper = new ArrayList(1); - wrapper.add(msg); - send(wrapper.iterator()); + private String srcAddressName() { + String name = null; + Channel c = channelRef.get(); + if (c != null) { + SocketAddress address = c.getLocalAddress(); + if (address != null) { + name = address.toString(); + } + } + return name; } - private void flushRequest(Channel channel, final MessageBatch requests) { - if (requests == null) - return; + @Override public String toString() { + return String.format("Netty client for connecting to %s", dstAddressPrefixedName); + } - pendings.incrementAndGet(); - ChannelFuture future = channel.write(requests); - future.addListener(new ChannelFutureListener() { - public void operationComplete(ChannelFuture future) - throws Exception { + /** + * Asynchronously establishes a Netty connection to the remote address, returning a Netty Channel on success. + */ + private class Connector implements Callable { - pendings.decrementAndGet(); - if (!future.isSuccess()) { - LOG.info( - "failed to send requests to " + remote_addr.toString() + ": ", future.getCause()); + private final InetSocketAddress address; + private final int connectionAttempt; - Channel channel = future.getChannel(); + public Connector(InetSocketAddress address, int connectionAttempt) { + this.address = address; + if (connectionAttempt < 1) { + throw new IllegalArgumentException("connection attempt must be >= 1 (you provided " + + connectionAttempt + ")"); + } + this.connectionAttempt = connectionAttempt; + } - if (null != channel) { - channel.close(); - channelRef.compareAndSet(channel, null); + @Override public Channel call() throws Exception { + LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); + Channel channel = null; + ChannelFuture future = bootstrap.connect(address); + future.awaitUninterruptibly(); + Channel current = future.getChannel(); + + if (future.isSuccess() && connectionEstablished(current)) { + channel = current; + LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), channel.toString(), + connectionAttempt); } - } else { - LOG.debug("{} request(s) sent", requests.size()); + else { + LOG.debug("failed to connect to {} [attempt {}]", address.toString(), connectionAttempt); + if (current != null) { + current.close(); } } - }); + return channel; } } +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java index 2499e65318f..5598a39fb6b 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Server.java @@ -37,15 +37,37 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +import org.jboss.netty.bootstrap.ServerBootstrap; +import org.jboss.netty.channel.Channel; +import org.jboss.netty.channel.ChannelFactory; +import org.jboss.netty.channel.group.ChannelGroup; +import org.jboss.netty.channel.group.DefaultChannelGroup; +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import backtype.storm.Config; +import backtype.storm.messaging.ConnectionWithStatus; +import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.TaskMessage; +import backtype.storm.metric.api.IStatefulObject; +import backtype.storm.utils.Utils; + +class Server extends ConnectionWithStatus implements IStatefulObject { -class Server implements IConnection { private static final Logger LOG = LoggerFactory.getLogger(Server.class); @SuppressWarnings("rawtypes") Map storm_conf; int port; + private final ConcurrentHashMap messagesEnqueued = new ConcurrentHashMap(); + private final AtomicInteger messagesDequeued = new AtomicInteger(0); + private final AtomicInteger[] pendingMessages; // Create multiple queues for incoming messages. The size equals the number of receiver threads. // For message which is sent to same task, it will be stored in the same queue to preserve the message order. @@ -59,7 +81,7 @@ class Server implements IConnection { private volatile HashMap taskToQueueId = null; int roundRobinQueueId; - boolean closing = false; + private volatile boolean closing = false; List closeMessage = Arrays.asList(new TaskMessage(-1, null)); @@ -73,8 +95,10 @@ class Server implements IConnection { taskToQueueId = new HashMap(); message_queue = new LinkedBlockingQueue[queueCount]; + pendingMessages = new AtomicInteger[queueCount]; for (int i = 0; i < queueCount; i++) { message_queue[i] = new LinkedBlockingQueue>(); + pendingMessages[i] = new AtomicInteger(0); } // Configure the server. @@ -149,13 +173,30 @@ private Integer getMessageQueueId(int task) { return queueId; } + private void addReceiveCount(String from, int amount) { + //This is possibly lossy in the case where a value is deleted + // because it has received no messages over the metrics collection + // period and new messages are starting to come in. This is + // because I don't want the overhead of a synchronize just to have + // the metric be absolutely perfect. + AtomicInteger i = messagesEnqueued.get(from); + if (i == null) { + i = new AtomicInteger(amount); + AtomicInteger prev = messagesEnqueued.putIfAbsent(from, i); + if (prev != null) { + prev.addAndGet(amount); + } + } else { + i.addAndGet(amount); + } + } + + /** * enqueue a received message - * @param message * @throws InterruptedException */ protected void enqueue(List msgs) throws InterruptedException { - if (null == msgs || msgs.size() == 0 || closing) { return; } @@ -170,11 +211,12 @@ protected void enqueue(List msgs) throws InterruptedException { ArrayList msgGroup = messageGroups[receiverId]; if (null != msgGroup) { message_queue[receiverId].put(msgGroup); + pendingMessages[receiverId].addAndGet(msgGroup.size()); } } } - public Iterator recv(int flags, int receiverId) { + public Iterator recv(int flags, int receiverId) { if (closing) { return closeMessage.iterator(); } @@ -184,18 +226,22 @@ public Iterator recv(int flags, int receiverId) { if ((flags & 0x01) == 0x01) { //non-blocking ret = message_queue[queueId].poll(); - } else { + } + else { try { ArrayList request = message_queue[queueId].take(); LOG.debug("request to be processed: {}", request); ret = request; - } catch (InterruptedException e) { + } + catch (InterruptedException e) { LOG.info("exception within msg receiving", e); ret = null; } } if (null != ret) { + messagesDequeued.addAndGet(ret.size()); + pendingMessages[queueId].addAndGet(0 - ret.size()); return ret.iterator(); } return null; @@ -230,14 +276,72 @@ public synchronized void close() { } public void send(int task, byte[] message) { - throw new RuntimeException("Server connection should not send any messages"); + throw new UnsupportedOperationException("Server connection should not send any messages"); } public void send(Iterator msgs) { - throw new RuntimeException("Server connection should not send any messages"); + throw new UnsupportedOperationException("Server connection should not send any messages"); } public String name() { return "Netty-server-localhost-" + port; } + + @Override + public Status status() { + if (closing) { + return Status.Closed; + } + else if (!connectionEstablished(allChannels)) { + return Status.Connecting; + } + else { + return Status.Ready; + } + } + + private boolean connectionEstablished(Channel channel) { + return channel != null && channel.isBound(); + } + + private boolean connectionEstablished(ChannelGroup allChannels) { + boolean allEstablished = true; + for (Channel channel : allChannels) { + if (!(connectionEstablished(channel))) { + allEstablished = false; + break; + } + } + return allEstablished; + } + + public Object getState() { + LOG.info("Getting metrics for server on port {}", port); + HashMap ret = new HashMap(); + ret.put("dequeuedMessages", messagesDequeued.getAndSet(0)); + ArrayList pending = new ArrayList(pendingMessages.length); + for (AtomicInteger p: pendingMessages) { + pending.add(p.get()); + } + ret.put("pending", pending); + HashMap enqueued = new HashMap(); + Iterator> it = messagesEnqueued.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry ent = it.next(); + //Yes we can delete something that is not 0 because of races, but that is OK for metrics + AtomicInteger i = ent.getValue(); + if (i.get() == 0) { + it.remove(); + } else { + enqueued.put(ent.getKey(), i.getAndSet(0)); + } + } + ret.put("enqueued", enqueued); + return ret; + } + + @Override public String toString() { + return String.format("Netty server listening on port %s", port); + } + } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java index e6e8b3dec89..73c50a18362 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java @@ -37,7 +37,7 @@ public ChannelPipeline getPipeline() throws Exception { // Encoder pipeline.addLast("encoder", new MessageEncoder()); // business logic. - pipeline.addLast("handler", new StormClientErrorHandler(client.name())); + pipeline.addLast("handler", new StormClientErrorHandler(client.dstAddressPrefixedName)); return pipeline; } diff --git a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj index ed5797d129e..04f2a4b81dc 100644 --- a/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj +++ b/storm-core/test/clj/backtype/storm/messaging/netty_unit_test.clj @@ -16,14 +16,36 @@ (ns backtype.storm.messaging.netty-unit-test (:use [clojure test]) (:import [backtype.storm.messaging TransportFactory]) - (:use [backtype.storm bootstrap testing util])) + (:use [backtype.storm bootstrap testing util]) + (:use [backtype.storm.daemon.worker :only [is-connection-ready]])) (bootstrap) (def port 6700) (def task 1) +;; In a "real" cluster (or an integration test), Storm itself would ensure that a topology's workers would only be +;; activated once all the workers' connections are ready. The tests in this file however launch Netty servers and +;; clients directly, and thus we must ensure manually that the server and the client connections are ready before we +;; commence testing. If we don't do this, then we will lose the first messages being sent between the client and the +;; server, which will fail the tests. +(defn- wait-until-ready + ([connections] + (do (log-message "Waiting until all Netty connections are ready...") + (wait-until-ready connections 0))) + ([connections waited-ms] + (let [interval-ms 10 + max-wait-ms 5000] + (if-not (every? is-connection-ready connections) + (if (<= waited-ms max-wait-ms) + (do + (Thread/sleep interval-ms) + (wait-until-ready connections (+ waited-ms interval-ms))) + (throw (RuntimeException. (str "Netty connections were not ready within " max-wait-ms " ms")))) + (log-message "All Netty connections are ready"))))) + (deftest test-basic + (log-message "Should send and receive a basic message") (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 @@ -36,6 +58,7 @@ context (TransportFactory/makeContext storm-conf) server (.bind context nil port) client (.connect context nil "localhost" port) + _ (wait-until-ready [server client]) _ (.send client task (.getBytes req_msg)) iter (.recv server 0 0) resp (.next iter)] @@ -46,6 +69,7 @@ (.term context))) (deftest test-large-msg + (log-message "Should send and receive a large message") (let [req_msg (apply str (repeat 2048000 'c')) storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 102400 @@ -58,6 +82,7 @@ context (TransportFactory/makeContext storm-conf) server (.bind context nil port) client (.connect context nil "localhost" port) + _ (wait-until-ready [server client]) _ (.send client task (.getBytes req_msg)) iter (.recv server 0 0) resp (.next iter)] @@ -67,38 +92,9 @@ (.close server) (.term context))) -(deftest test-server-delayed - (let [req_msg (String. "0123456789abcdefghijklmnopqrstuvwxyz") - storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" - STORM-MESSAGING-NETTY-BUFFER-SIZE 1024 - STORM-MESSAGING-NETTY-MAX-RETRIES 10 - STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 - STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 - STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 - STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 - } - context (TransportFactory/makeContext storm-conf) - client (.connect context nil "localhost" port) - - server (Thread. - (fn [] - (Thread/sleep 1000) - (let [server (.bind context nil port) - iter (.recv server 0 0) - resp (.next iter)] - (is (= task (.task resp))) - (is (= req_msg (String. (.message resp)))) - (.close server) - ))) - _ (.start server) - _ (.send client task (.getBytes req_msg)) - ] - (.close client) - (.join server) - (.term context))) - (deftest test-batch - (let [storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" + (let [num-messages 100000 + storm-conf {STORM-MESSAGING-TRANSPORT "backtype.storm.messaging.netty.Context" STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000 STORM-MESSAGING-NETTY-MAX-RETRIES 10 STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 @@ -106,23 +102,25 @@ STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1 STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1 } + _ (log-message "Should send and receive many messages (testing with " num-messages " messages)") context (TransportFactory/makeContext storm-conf) server (.bind context nil port) - client (.connect context nil "localhost" port)] - (doseq [num (range 1 100000)] + client (.connect context nil "localhost" port) + _ (wait-until-ready [server client])] + (doseq [num (range 1 num-messages)] (let [req_msg (str num)] (.send client task (.getBytes req_msg)))) (let [resp (ArrayList.) received (atom 0)] - (while (< @received (- 100000 1)) + (while (< @received (- num-messages 1)) (let [iter (.recv server 0 0)] (while (.hasNext iter) (let [msg (.next iter)] (.add resp msg) (swap! received inc) )))) - (doseq [num (range 1 100000)] + (doseq [num (range 1 num-messages)] (let [req_msg (str num) resp_msg (String. (.message (.get resp (- num 1))))] (is (= req_msg resp_msg))))) diff --git a/storm-core/test/clj/backtype/storm/worker_test.clj b/storm-core/test/clj/backtype/storm/worker_test.clj new file mode 100644 index 00000000000..f09baefc862 --- /dev/null +++ b/storm-core/test/clj/backtype/storm/worker_test.clj @@ -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. +(ns backtype.storm.worker-test + (:use [clojure test]) + (:import [backtype.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status]) + (:import [org.mockito Mockito]) + (:use [backtype.storm bootstrap testing]) + (:use [backtype.storm.daemon common]) + + (:require [backtype.storm.daemon [worker :as worker]]) + ) + +(bootstrap) + +(deftest test-worker-is-connection-ready + (let [connection (Mockito/mock ConnectionWithStatus)] + (. (Mockito/when (.status connection)) thenReturn ConnectionWithStatus$Status/Ready) + (is (= true (worker/is-connection-ready connection))) + + (. (Mockito/when (.status connection)) thenReturn ConnectionWithStatus$Status/Connecting) + (is (= false (worker/is-connection-ready connection))) + + (. (Mockito/when (.status connection)) thenReturn ConnectionWithStatus$Status/Closed) + (is (= false (worker/is-connection-ready connection))) + )) \ No newline at end of file From 81016c2ed7222da99138bc9971e335533d4cb518 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 16 Feb 2015 10:01:27 +0100 Subject: [PATCH 04/89] Track how many messages are being dropped when a connection is unavailable Signed-off-by: P. Taylor Goetz --- storm-core/src/jvm/backtype/storm/messaging/netty/Client.java | 1 + 1 file changed, 1 insertion(+) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index ae8a8d52d55..f275ef6f0ac 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -449,6 +449,7 @@ private void handleMessagesWhenConnectionIsUnavailable(Iterator msg private void dropPendingMessages(Iterator msgs) { // We consume the iterator by traversing and thus "emptying" it. int msgCount = iteratorSize(msgs); + messagesLost.getAndAdd(msgCount); LOG.error("dropping {} pending message(s) destined for {}", msgCount, dstAddressPrefixedName); } From 97a76fc896de508f015dbe32f1473ddbf10d736b Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 16 Feb 2015 10:03:07 +0100 Subject: [PATCH 05/89] Clarify name of method for dropping messages Signed-off-by: P. Taylor Goetz --- .../src/jvm/backtype/storm/messaging/netty/Client.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index f275ef6f0ac..77c20c8de43 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -443,14 +443,14 @@ private boolean hasMessages(Iterator msgs) { */ private void handleMessagesWhenConnectionIsUnavailable(Iterator msgs) { LOG.error("connection to {} is unavailable", dstAddressPrefixedName); - dropPendingMessages(msgs); + dropMessages(msgs); } - private void dropPendingMessages(Iterator msgs) { + private void dropMessages(Iterator msgs) { // We consume the iterator by traversing and thus "emptying" it. int msgCount = iteratorSize(msgs); messagesLost.getAndAdd(msgCount); - LOG.error("dropping {} pending message(s) destined for {}", msgCount, dstAddressPrefixedName); + LOG.error("dropping {} message(s) destined for {}", msgCount, dstAddressPrefixedName); } private int iteratorSize(Iterator msgs) { From 9138d9fc255639b4d0d43657379ce467591e8ef2 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Mon, 16 Feb 2015 10:07:35 +0100 Subject: [PATCH 06/89] Change log level for intentionally dropping messages from WARN to ERROR This change makes the log level for dropping messages consistent in Client.java. Signed-off-by: P. Taylor Goetz --- storm-core/src/jvm/backtype/storm/messaging/netty/Client.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 77c20c8de43..afce49688f6 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -370,7 +370,7 @@ public void send(int taskId, byte[] payload) { public synchronized void send(Iterator msgs) { if (closing) { int numMessages = iteratorSize(msgs); - LOG.warn("discarding {} messages because the Netty client to {} is being closed", numMessages, + LOG.error("discarding {} messages because the Netty client to {} is being closed", numMessages, dstAddressPrefixedName); return; } @@ -487,7 +487,7 @@ public void operationComplete(ChannelFuture future) throws Exception { messagesSent.getAndAdd(batch.size()); } else { - LOG.warn("failed to send {} messages to {}: {}", numMessages, dstAddressPrefixedName, + LOG.error("failed to send {} messages to {}: {}", numMessages, dstAddressPrefixedName, future.getCause()); closeChannelAndReconnect(future.getChannel()); messagesLost.getAndAdd(numMessages); From 6b06d8468ff5e743fb12b85dd84fe0931041c2c3 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Tue, 24 Feb 2015 18:18:43 -0500 Subject: [PATCH 07/89] add STORM-329 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5839b7a27af..7dd058d130c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.4 + * STORM-329: fix cascading Storm failure by improving reconnection strategy and buffering messages * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. ## 0.9.3-rc2 From e63fb2af9086e2b2e688662ca42a4b4d0112274b Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Mon, 2 Mar 2015 16:06:58 -0800 Subject: [PATCH 08/89] STORM-693: when bolt fails to write tuple, it should report error instead of silently acking. Signed-off-by: P. Taylor Goetz --- .../storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java | 7 +++---- .../src/test/storm/kafka/bolt/KafkaBoltTest.java | 2 +- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java b/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java index b6c3de487c4..cf169dc4015 100644 --- a/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java +++ b/external/storm-kafka/src/jvm/storm/kafka/bolt/KafkaBolt.java @@ -101,11 +101,10 @@ public void execute(Tuple input) { } else { LOG.warn("skipping key = " + key + ", topic selector returned null."); } - } catch (Exception ex) { - LOG.error("Could not send message with key = " + key - + " and value = " + message + " to topic = " + topic, ex); - } finally { collector.ack(input); + } catch (Exception ex) { + collector.reportError(ex); + collector.fail(input); } } diff --git a/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java b/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java index 2a56f84ecef..c30cba132bc 100644 --- a/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java +++ b/external/storm-kafka/src/test/storm/kafka/bolt/KafkaBoltTest.java @@ -143,7 +143,7 @@ public void executeWithBrokerDown() throws Exception { String message = "value-234"; Tuple tuple = generateTestTuple(message); bolt.execute(tuple); - verify(collector).ack(tuple); + verify(collector).fail(tuple); } From 92836de540ec8ab90d7591b96ba02126e80b5c3a Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 10:59:56 -0400 Subject: [PATCH 09/89] add STORM-693 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7dd058d130c..1432f041044 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.4 + * STORM-693: when kafka bolt fails to write tuple, it should report error instead of silently acking. * STORM-329: fix cascading Storm failure by improving reconnection strategy and buffering messages * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. From c19e482b70f18d690ad165c78551860506486095 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Fri, 20 Feb 2015 11:56:22 -0800 Subject: [PATCH 10/89] STORM-682: supervisor should handle worker state corruption gracefully. Signed-off-by: P. Taylor Goetz --- storm-core/src/clj/backtype/storm/daemon/supervisor.clj | 7 +++++-- .../src/jvm/backtype/storm/utils/VersionedStore.java | 9 +++++---- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj index 582c3df69ad..f5eaf42c7ee 100644 --- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj @@ -93,8 +93,11 @@ (defn read-worker-heartbeat [conf id] (let [local-state (worker-state conf id)] - (.get local-state LS-WORKER-HEARTBEAT) - )) + (try + (.get local-state LS-WORKER-HEARTBEAT) + (catch IOException e + (log-warn e "Failed to read local heartbeat for workerId : " id ",Ignoring exception.") + nil)))) (defn my-worker-ids [conf] diff --git a/storm-core/src/jvm/backtype/storm/utils/VersionedStore.java b/storm-core/src/jvm/backtype/storm/utils/VersionedStore.java index b40aa6050df..07ce5a8d92b 100644 --- a/storm-core/src/jvm/backtype/storm/utils/VersionedStore.java +++ b/storm-core/src/jvm/backtype/storm/utils/VersionedStore.java @@ -94,13 +94,14 @@ public void failVersion(String path) throws IOException { public void deleteVersion(long version) throws IOException { File versionFile = new File(versionPath(version)); File tokenFile = new File(tokenPath(version)); - + + if(tokenFile.exists()) { + FileUtils.forceDelete(tokenFile); + } + if(versionFile.exists()) { FileUtils.forceDelete(versionFile); } - if(tokenFile.exists()) { - FileUtils.forceDelete(tokenFile); - } } public void succeedVersion(String path) throws IOException { From f0de11a20fe2f20dc1dc2f485549e0dc342f8680 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 11:05:30 -0400 Subject: [PATCH 11/89] add STORM-682 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1432f041044..888ccc1ce08 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.4 + * STORM-682: supervisor should handle worker state corruption gracefully. * STORM-693: when kafka bolt fails to write tuple, it should report error instead of silently acking. * STORM-329: fix cascading Storm failure by improving reconnection strategy and buffering messages * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. From 835a410c879dc1eb02d9670410f65fe0be6f28c6 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Wed, 14 Jan 2015 12:27:35 -0800 Subject: [PATCH 12/89] STORM-559: ZkHosts in README should use 2181 as port. Signed-off-by: P. Taylor Goetz --- external/storm-kafka/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/storm-kafka/README.md b/external/storm-kafka/README.md index 02d9767a10c..7442a0eb4e1 100644 --- a/external/storm-kafka/README.md +++ b/external/storm-kafka/README.md @@ -18,7 +18,7 @@ Kafka's zookeeper's entries to track brokerHost -> partition mapping. You can in public ZkHosts(String brokerZkStr, String brokerZkPath) public ZkHosts(String brokerZkStr) ``` -Where brokerZkStr is just ip:port e.g. localhost:9092. brokerZkPath is the root directory under which all the topics and +Where brokerZkStr is just ip:port e.g. localhost:2181. brokerZkPath is the root directory under which all the topics and partition information is stored. by Default this is /brokers which is what default kafka implementation uses. By default the broker-partition mapping is refreshed every 60 seconds from zookeeper. If you want to change it you From 30e0be8616c89cb1f8a51fcf462f76a075e6e964 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 11:11:16 -0400 Subject: [PATCH 13/89] add STORM-559 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 888ccc1ce08..ad81e435792 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.4 + * STORM-559: ZkHosts in README should use 2181 as port. * STORM-682: supervisor should handle worker state corruption gracefully. * STORM-693: when kafka bolt fails to write tuple, it should report error instead of silently acking. * STORM-329: fix cascading Storm failure by improving reconnection strategy and buffering messages From b1bbacb7134d17ff47c2e8b8857a66244a4d1d4f Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 11:28:11 -0400 Subject: [PATCH 14/89] add missing import in supervisor.clj --- storm-core/src/clj/backtype/storm/daemon/supervisor.clj | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj index f5eaf42c7ee..df7330a2627 100644 --- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj @@ -15,7 +15,7 @@ ;; limitations under the License. (ns backtype.storm.daemon.supervisor (:import [backtype.storm.scheduler ISupervisor] - [java.net JarURLConnection] + [java.io IOException] [java.net URI]) (:use [backtype.storm bootstrap]) (:use [backtype.storm.daemon common]) From edf596bac8feab0c8721f7de94474e5549858355 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 12:21:38 -0400 Subject: [PATCH 15/89] [maven-release-plugin] prepare release v0.9.4 --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index cd7becba2fa..338e54886ec 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index e687da8a8da..62ce62d018d 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 9e8d0c4fff0..3251bfac8a4 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 4d4a871b1ba..9ba7e5e31d2 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/pom.xml b/pom.xml index 6f4d613dbb2..a94193157f2 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.4-SNAPSHOT + 0.9.4 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.4 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index b5b2098158d..73a93cedf96 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 496f6b62ccb..a5f99a88785 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index cc0a6f9f043..79f098165cf 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 4108a066b6b..3cee9242954 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml org.apache.storm From 48d10e20eb3c750fc41fcf0bef3d49501cf6d5a4 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 12:21:45 -0400 Subject: [PATCH 16/89] [maven-release-plugin] prepare for next development iteration --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 338e54886ec..5aef022af6c 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index 62ce62d018d..ca28c507f9a 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 3251bfac8a4..b44972308cd 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 9ba7e5e31d2..2a368bee888 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index a94193157f2..2b446c1b885 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.4 + 0.9.5-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.4 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 73a93cedf96..e1e13b07bc0 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index a5f99a88785..f40dedfd788 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 79f098165cf..21d1a809a4a 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 3cee9242954..ea21d7c0c92 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml org.apache.storm From 41f44f9914d4f27d0db3f211a85f88301533f09b Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 13:59:39 -0400 Subject: [PATCH 17/89] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 48d10e20eb3c750fc41fcf0bef3d49501cf6d5a4. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 5aef022af6c..338e54886ec 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index ca28c507f9a..62ce62d018d 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index b44972308cd..3251bfac8a4 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 2a368bee888..9ba7e5e31d2 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/pom.xml b/pom.xml index 2b446c1b885..a94193157f2 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.5-SNAPSHOT + 0.9.4 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.4 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e1e13b07bc0..73a93cedf96 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index f40dedfd788..a5f99a88785 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 21d1a809a4a..79f098165cf 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index ea21d7c0c92..3cee9242954 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.4 ../../pom.xml org.apache.storm From 233603c3cbd729fdfabd2759bfa7705811996aa4 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 14:00:11 -0400 Subject: [PATCH 18/89] Revert "[maven-release-plugin] prepare release v0.9.4" This reverts commit edf596bac8feab0c8721f7de94474e5549858355. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 338e54886ec..cd7becba2fa 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index 62ce62d018d..e687da8a8da 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 3251bfac8a4..9e8d0c4fff0 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 9ba7e5e31d2..4d4a871b1ba 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index a94193157f2..6f4d613dbb2 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.4 + 0.9.4-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.4 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 73a93cedf96..b5b2098158d 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index a5f99a88785..496f6b62ccb 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 79f098165cf..cc0a6f9f043 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 3cee9242954..4108a066b6b 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-SNAPSHOT ../../pom.xml org.apache.storm From 61e1b5c3e226122143c91bbd7527b605f8ed8727 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 14:05:51 -0400 Subject: [PATCH 19/89] add Apache license header to ConnectionWithStatus.java --- .../storm/messaging/ConnectionWithStatus.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java b/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java index 38abc19edfc..37981ca9095 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java +++ b/storm-core/src/jvm/backtype/storm/messaging/ConnectionWithStatus.java @@ -1,3 +1,20 @@ +/** + * 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 backtype.storm.messaging; public abstract class ConnectionWithStatus implements IConnection { From 00091d7952681a39281aa171adfad133a5e26330 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 14:13:57 -0400 Subject: [PATCH 20/89] [maven-release-plugin] prepare release v0.9.4 --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index cd7becba2fa..338e54886ec 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index e687da8a8da..62ce62d018d 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 9e8d0c4fff0..3251bfac8a4 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 4d4a871b1ba..9ba7e5e31d2 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/pom.xml b/pom.xml index 6f4d613dbb2..a94193157f2 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.4-SNAPSHOT + 0.9.4 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.4 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index b5b2098158d..73a93cedf96 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 496f6b62ccb..a5f99a88785 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index cc0a6f9f043..79f098165cf 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 4108a066b6b..3cee9242954 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-SNAPSHOT + 0.9.4 ../../pom.xml org.apache.storm From c248dd98f681e56ea658a0102d204853f6803adc Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 18 Mar 2015 14:14:04 -0400 Subject: [PATCH 21/89] [maven-release-plugin] prepare for next development iteration --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 338e54886ec..5aef022af6c 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index 62ce62d018d..ca28c507f9a 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 3251bfac8a4..b44972308cd 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 9ba7e5e31d2..2a368bee888 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index a94193157f2..2b446c1b885 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.4 + 0.9.5-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.4 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 73a93cedf96..e1e13b07bc0 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index a5f99a88785..f40dedfd788 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 79f098165cf..21d1a809a4a 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 3cee9242954..ea21d7c0c92 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.5-SNAPSHOT ../../pom.xml org.apache.storm From ed5a51f679d38281a62f2a9ac15c13853bc4c577 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 6 Apr 2015 23:53:31 +0900 Subject: [PATCH 22/89] fix storm.cmd to evaluate 'shift' correctly with 'storm jar' --- bin/storm.cmd | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/storm.cmd b/bin/storm.cmd index 0cf6b526abd..bfb4d932db6 100644 --- a/bin/storm.cmd +++ b/bin/storm.cmd @@ -70,11 +70,11 @@ set CLASSPATH=%CLASSPATH%;%2 set CLASS=%3 set args=%4 - shift + goto start :start + shift if [%4] == [] goto done set args=%args% %4 - shift goto start :done From a480eab7bac6a3f754da223a30e495ac5480eeba Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 7 May 2015 11:26:11 -0400 Subject: [PATCH 23/89] update changelog for version 0.9.5 --- CHANGELOG.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index ad81e435792..94e0f4c5f9f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,6 @@ +## 0.9.5 + * STORM-745: fix storm.cmd to evaluate 'shift' correctly with 'storm jar' + ## 0.9.4 * STORM-559: ZkHosts in README should use 2181 as port. * STORM-682: supervisor should handle worker state corruption gracefully. From 4d550c0f085b86a0c305bd136bb1ddb7cb833d21 Mon Sep 17 00:00:00 2001 From: Andrew Montalenti Date: Thu, 23 Apr 2015 16:41:39 -0400 Subject: [PATCH 24/89] add support for "error" command in ShellSpout --- storm-core/src/jvm/backtype/storm/spout/ShellSpout.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java index c79d175e6e5..ece11eeca45 100644 --- a/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java +++ b/storm-core/src/jvm/backtype/storm/spout/ShellSpout.java @@ -154,6 +154,8 @@ private void querySubprocess() { return; } else if (command.equals("log")) { handleLog(shellMsg); + } else if (command.equals("error")) { + handleError(shellMsg.getMsg()); } else if (command.equals("emit")) { String stream = shellMsg.getStream(); Long task = shellMsg.getTask(); @@ -206,6 +208,10 @@ private void handleLog(ShellMsg shellMsg) { } } + private void handleError(String msg) { + _collector.reportError(new Exception("Shell Process Exception: " + msg)); + } + @Override public void activate() { LOG.info("Start checking heartbeat..."); From 97de0853f252a173eaf13eed9a54e47009318f66 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 7 May 2015 12:07:43 -0400 Subject: [PATCH 25/89] update changelog for STORM-796 --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 94e0f4c5f9f..f86558ddecf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.5 + * STORM-796: Add support for "error" command in ShellSpout * STORM-745: fix storm.cmd to evaluate 'shift' correctly with 'storm jar' ## 0.9.4 From ed8ab3ec194f19c75fc2f5c000609204f04b50e8 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Thu, 28 May 2015 20:42:05 +0100 Subject: [PATCH 26/89] Simplified the flow and removed the lock that was causing the deadlock --- .../storm/messaging/netty/Client.java | 376 ++++++------------ .../storm/messaging/netty/MessageBatch.java | 4 + 2 files changed, 120 insertions(+), 260 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index afce49688f6..4e970356dd9 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -6,9 +6,9 @@ * 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. @@ -18,10 +18,14 @@ package backtype.storm.messaging.netty; import backtype.storm.Config; -import backtype.storm.messaging.IConnection; +import backtype.storm.messaging.ConnectionWithStatus; import backtype.storm.messaging.TaskMessage; +import backtype.storm.metric.api.IStatefulObject; import backtype.storm.utils.StormBoundedExponentialBackoffRetry; import backtype.storm.utils.Utils; +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import org.jboss.netty.bootstrap.ClientBootstrap; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; @@ -29,36 +33,21 @@ import org.jboss.netty.channel.ChannelFutureListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.HashMap; -import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import com.google.common.util.concurrent.*; -import org.jboss.netty.bootstrap.ClientBootstrap; -import org.jboss.netty.channel.Channel; -import org.jboss.netty.channel.ChannelFactory; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelFutureListener; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import backtype.storm.Config; -import backtype.storm.messaging.ConnectionWithStatus; -import backtype.storm.metric.api.IStatefulObject; -import backtype.storm.messaging.TaskMessage; -import backtype.storm.utils.StormBoundedExponentialBackoffRetry; -import backtype.storm.utils.Utils; +import static com.google.common.base.Preconditions.checkState; /** * A Netty client for sending task messages to a remote destination (Netty server). @@ -81,9 +70,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private static final String PREFIX = "Netty-Client-"; private static final long NO_DELAY_MS = 0L; private static final long MINIMUM_INITIAL_DELAY_MS = 30000L; - private static final long PENDING_MESSAGES_FLUSH_TIMEOUT_MS = 600000L; - private static final long PENDING_MESSAGES_FLUSH_INTERVAL_MS = 1000L; - private static final long DISTANT_FUTURE_TIME_MS = Long.MAX_VALUE; private final StormBoundedExponentialBackoffRetry retryPolicy; private final ClientBootstrap bootstrap; @@ -93,7 +79,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { /** * The channel used for all write operations from this client to the remote destination. */ - private final AtomicReference channelRef = new AtomicReference(null); + private final AtomicReference channelRef = new AtomicReference(); /** @@ -126,29 +112,12 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private final AtomicLong pendingMessages = new AtomicLong(0); + /** * This flag is set to true if and only if a client instance is being closed. */ private volatile boolean closing = false; - /** - * When set to true, then the background flusher thread will flush any pending messages on its next run. - */ - private final AtomicBoolean backgroundFlushingEnabled = new AtomicBoolean(false); - - /** - * The absolute time (in ms) when the next background flush should be performed. - * - * Note: The flush operation will only be performed if backgroundFlushingEnabled is true, too. - */ - private final AtomicLong nextBackgroundFlushTimeMs = new AtomicLong(DISTANT_FUTURE_TIME_MS); - - /** - * The time interval (in ms) at which the background flusher thread will be run to check for any pending messages - * to be flushed. - */ - private final int flushCheckIntervalMs; - /** * How many messages should be batched together before sending them to the remote destination. * @@ -156,36 +125,38 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private final int messageBatchSize; - private MessageBatch messageBatch = null; private final ListeningScheduledExecutorService scheduler; protected final Map stormConf; + private AtomicReference pendingMessageBatch; + @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { closing = false; this.stormConf = stormConf; - this.scheduler = MoreExecutors.listeningDecorator(scheduler); + this.scheduler = MoreExecutors.listeningDecorator(scheduler); int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize); messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); - flushCheckIntervalMs = Utils.getInt(stormConf.get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); + int flushCheckIntervalMs = Utils.getInt(stormConf.get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES)); int minWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); int maxWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); retryPolicy = new StormBoundedExponentialBackoffRetry(minWaitMs, maxWaitMs, maxReconnectionAttempts); + pendingMessageBatch = new AtomicReference(new MessageBatch(messageBatchSize)); + // Initiate connection to remote destination bootstrap = createClientBootstrap(factory, bufferSize); dstAddress = new InetSocketAddress(host, port); dstAddressPrefixedName = prefixedName(dstAddress); - connect(NO_DELAY_MS); - + scheduleConnect(NO_DELAY_MS); + // Launch background flushing thread - pauseBackgroundFlushing(); long initialDelayMs = Math.min(MINIMUM_INITIAL_DELAY_MS, maxWaitMs * maxReconnectionAttempts); scheduler.scheduleWithFixedDelay(createBackgroundFlusher(), initialDelayMs, flushCheckIntervalMs, - TimeUnit.MILLISECONDS); + TimeUnit.MILLISECONDS); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -203,114 +174,34 @@ private String prefixedName(InetSocketAddress dstAddress) { } return ""; } - + private Runnable createBackgroundFlusher() { return new Runnable() { @Override - public void run() { - if(!closing && backgroundFlushingEnabled.get() && nowMillis() > nextBackgroundFlushTimeMs.get()) { - LOG.debug("flushing {} pending messages to {} in background", messageBatch.size(), - dstAddressPrefixedName); + public void run() { + if (!closing) { + LOG.debug("flushing pending messages to {} in background", dstAddressPrefixedName); flushPendingMessages(); } } }; } - private void pauseBackgroundFlushing() { - backgroundFlushingEnabled.set(false); - } - - private void resumeBackgroundFlushing() { - backgroundFlushingEnabled.set(true); - } - - private synchronized void flushPendingMessages() { - Channel channel = channelRef.get(); - if (containsMessages(messageBatch)) { - if (connectionEstablished(channel)) { - if (channel.isWritable()) { - pauseBackgroundFlushing(); - MessageBatch toBeFlushed = messageBatch; - flushMessages(channel, toBeFlushed); - messageBatch = null; - } - else if (closing) { - // Ensure background flushing is enabled so that we definitely have a chance to re-try the flush - // operation in case the client is being gracefully closed (where we have a brief time window where - // the client will wait for pending messages to be sent). - resumeBackgroundFlushing(); - } - } - else { - closeChannelAndReconnect(channel); - } - } - } - - private long nowMillis() { - return System.currentTimeMillis(); + private void flushPendingMessages() { + Channel channel = channelRef.get(); + if (connectionEstablished(channel)) { + MessageBatch toFlush = pendingMessageBatch.getAndSet(new MessageBatch(messageBatchSize)); + flushMessages(channel, toFlush); + } else { + closeChannelAndReconnect(channel); + } } /** * We will retry connection with exponential back-off policy */ - private synchronized void connect(long delayMs) { - try { - if (closing) { - return; - } - - if (connectionEstablished(channelRef.get())) { - return; - } - - connectionAttempts.getAndIncrement(); - if (reconnectingAllowed()) { - totalConnectionAttempts.getAndIncrement(); - LOG.info("connection attempt {} to {} scheduled to run in {} ms", connectionAttempts.get(), - dstAddressPrefixedName, delayMs); - ListenableFuture channelFuture = scheduler.schedule( - new Connector(dstAddress, connectionAttempts.get()), delayMs, TimeUnit.MILLISECONDS); - Futures.addCallback(channelFuture, new FutureCallback() { - @Override public void onSuccess(Channel result) { - if (connectionEstablished(result)) { - setChannel(result); - LOG.info("connection established to {}", dstAddressPrefixedName); - connectionAttempts.set(0); - } - else { - reconnectAgain(new RuntimeException("Returned channel was actually not established")); - } - } - - @Override public void onFailure(Throwable t) { - reconnectAgain(t); - } - - private void reconnectAgain(Throwable t) { - String baseMsg = String.format("connection attempt %s to %s failed", connectionAttempts, - dstAddressPrefixedName); - String failureMsg = (t == null)? baseMsg : baseMsg + ": " + t.toString(); - LOG.error(failureMsg); - long nextDelayMs = retryPolicy.getSleepTimeMs(connectionAttempts.get(), 0); - connect(nextDelayMs); - } - }); - } - else { - close(); - throw new RuntimeException("Giving up to connect to " + dstAddressPrefixedName + " after " + - connectionAttempts + " failed attempts"); - } - } - catch (Exception e) { - throw new RuntimeException("Failed to connect to " + dstAddressPrefixedName, e); - } - } - - private void setChannel(Channel channel) { - channelRef.set(channel); + private void scheduleConnect(long delayMs) { + scheduler.schedule(new Connector(dstAddress), delayMs, TimeUnit.MILLISECONDS); } private boolean reconnectingAllowed() { @@ -336,11 +227,9 @@ private boolean connectionEstablished(Channel channel) { public Status status() { if (closing) { return Status.Closed; - } - else if (!connectionEstablished(channelRef.get())) { + } else if (!connectionEstablished(channelRef.get())) { return Status.Connecting; - } - else { + } else { return Status.Ready; } } @@ -367,14 +256,14 @@ public void send(int taskId, byte[] payload) { * Enqueue task messages to be sent to the remote destination (cf. `host` and `port`). */ @Override - public synchronized void send(Iterator msgs) { + public void send(Iterator msgs) { if (closing) { int numMessages = iteratorSize(msgs); LOG.error("discarding {} messages because the Netty client to {} is being closed", numMessages, - dstAddressPrefixedName); + dstAddressPrefixedName); return; } - + if (!hasMessages(msgs)) { return; } @@ -387,52 +276,26 @@ public synchronized void send(Iterator msgs) { return; } + MessageBatch toSend = pendingMessageBatch.getAndSet(new MessageBatch(messageBatchSize)); + // Collect messages into batches (to optimize network throughput), then flush them. while (msgs.hasNext()) { TaskMessage message = msgs.next(); - if (messageBatch == null) { - messageBatch = new MessageBatch(messageBatchSize); - } - - messageBatch.add(message); - if (messageBatch.isFull()) { - MessageBatch toBeFlushed = messageBatch; - flushMessages(channel, toBeFlushed); - messageBatch = null; + toSend.add(message); + if (toSend.isFull()) { + flushMessages(channel, toSend); + toSend = new MessageBatch(messageBatchSize); } } // Handle any remaining messages in case the "last" batch was not full. - if (containsMessages(messageBatch)) { - if (connectionEstablished(channel) && channel.isWritable()) { - // We can write to the channel, so we flush the remaining messages immediately to minimize latency. - pauseBackgroundFlushing(); - MessageBatch toBeFlushed = messageBatch; - messageBatch = null; - flushMessages(channel, toBeFlushed); - } - else { - // We cannot write to the channel, which means Netty's internal write buffer is full. - // In this case, we buffer the remaining messages and wait for the next messages to arrive. - // - // Background: - // Netty 3.x maintains an internal write buffer with a high water mark for each channel (default: 64K). - // This represents the amount of data waiting to be flushed to operating system buffers. If the - // outstanding data exceeds this value then the channel is set to non-writable. When this happens, a - // INTEREST_CHANGED channel event is triggered. Netty sets the channel to writable again once the data - // has been flushed to the system buffers. - // - // See http://stackoverflow.com/questions/14049260 - resumeBackgroundFlushing(); - nextBackgroundFlushTimeMs.set(nowMillis() + flushCheckIntervalMs); - } - } - - } + flushMessages(channel, toSend); + + } private boolean hasMessages(Iterator msgs) { return msgs != null && msgs.hasNext(); - } + } /** * We will drop pending messages and let at-least-once message replay kick in. @@ -463,32 +326,28 @@ private int iteratorSize(Iterator msgs) { } return size; } - + /** * Asynchronously writes the message batch to the channel. - * + * * If the write operation fails, then we will close the channel and trigger a reconnect. */ - private synchronized void flushMessages(Channel channel, final MessageBatch batch) { + private void flushMessages(Channel channel, final MessageBatch batch) { if (!containsMessages(batch)) { return; } final int numMessages = batch.size(); - pendingMessages.getAndAdd(numMessages); LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString()); ChannelFuture future = channel.write(batch); future.addListener(new ChannelFutureListener() { - public void operationComplete(ChannelFuture future) throws Exception { - pendingMessages.getAndAdd(0 - numMessages); if (future.isSuccess()) { LOG.debug("sent {} messages to {}", numMessages, dstAddressPrefixedName); messagesSent.getAndAdd(batch.size()); - } - else { + } else { LOG.error("failed to send {} messages to {}: {}", numMessages, dstAddressPrefixedName, - future.getCause()); + future.getCause()); closeChannelAndReconnect(future.getChannel()); messagesLost.getAndAdd(numMessages); } @@ -496,20 +355,20 @@ public void operationComplete(ChannelFuture future) throws Exception { }); } - - private synchronized void closeChannelAndReconnect(Channel channel) { - if (channel != null) { + + private void closeChannelAndReconnect(Channel channel) { + if (channel != null) { channel.close(); if (channelRef.compareAndSet(channel, null)) { - connect(NO_DELAY_MS); + scheduleConnect(NO_DELAY_MS); } - } - } - + } + } + private boolean containsMessages(MessageBatch batch) { return batch != null && !batch.isEmpty(); } - + /** * Gracefully close this client. * @@ -523,36 +382,14 @@ public void close() { // Set closing to true to prevent any further reconnection attempts. closing = true; flushPendingMessages(); - waitForPendingMessagesToBeSent(); closeChannel(); } } - - private synchronized void waitForPendingMessagesToBeSent() { - LOG.info("waiting up to {} ms to send {} pending messages to {}", - PENDING_MESSAGES_FLUSH_TIMEOUT_MS, pendingMessages.get(), dstAddressPrefixedName); - long totalPendingMsgs = pendingMessages.get(); - long startMs = nowMillis(); - while (pendingMessages.get() != 0) { - try { - long deltaMs = nowMillis() - startMs; - if (deltaMs > PENDING_MESSAGES_FLUSH_TIMEOUT_MS) { - LOG.error("failed to send all pending messages to {} within timeout, {} of {} messages were not " + - "sent", dstAddressPrefixedName, pendingMessages.get(), totalPendingMsgs); - break; - } - Thread.sleep(PENDING_MESSAGES_FLUSH_INTERVAL_MS); - } - catch (InterruptedException e) { - break; - } - } - - } - private synchronized void closeChannel() { - if (channelRef.get() != null) { - channelRef.get().close(); + private void closeChannel() { + Channel channel = channelRef.get(); + if (channel != null) { + channel.close(); LOG.debug("channel to {} closed", dstAddressPrefixedName); } } @@ -575,9 +412,9 @@ public Object getState() { private String srcAddressName() { String name = null; - Channel c = channelRef.get(); - if (c != null) { - SocketAddress address = c.getLocalAddress(); + Channel channel = channelRef.get(); + if (channel != null) { + SocketAddress address = channel.getLocalAddress(); if (address != null) { name = address.toString(); } @@ -585,47 +422,66 @@ private String srcAddressName() { return name; } - @Override public String toString() { + @Override + public String toString() { return String.format("Netty client for connecting to %s", dstAddressPrefixedName); } /** - * Asynchronously establishes a Netty connection to the remote address, returning a Netty Channel on success. + * Asynchronously establishes a Netty connection to the remote address */ - private class Connector implements Callable { + private class Connector implements Runnable { private final InetSocketAddress address; - private final int connectionAttempt; - public Connector(InetSocketAddress address, int connectionAttempt) { + public Connector(InetSocketAddress address) { this.address = address; - if (connectionAttempt < 1) { - throw new IllegalArgumentException("connection attempt must be >= 1 (you provided " + - connectionAttempt + ")"); - } - this.connectionAttempt = connectionAttempt; } - @Override public Channel call() throws Exception { - LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); - Channel channel = null; - ChannelFuture future = bootstrap.connect(address); - future.awaitUninterruptibly(); - Channel current = future.getChannel(); - - if (future.isSuccess() && connectionEstablished(current)) { - channel = current; - LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), channel.toString(), - connectionAttempt); + private void reconnectAgain(Throwable t) { + String baseMsg = String.format("connection attempt %s to %s failed", connectionAttempts, + dstAddressPrefixedName); + String failureMsg = (t == null) ? baseMsg : baseMsg + ": " + t.toString(); + LOG.error(failureMsg); + long nextDelayMs = retryPolicy.getSleepTimeMs(connectionAttempts.get(), 0); + scheduleConnect(nextDelayMs); + } + + + @Override + public void run() { + try { + if (reconnectingAllowed()) { + int connectionAttempt = connectionAttempts.getAndIncrement(); + totalConnectionAttempts.getAndIncrement(); + + LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); + ChannelFuture future = bootstrap.connect(address); + future.awaitUninterruptibly(); + Channel newChannel = future.getChannel(); + + if (future.isSuccess() && connectionEstablished(newChannel)) { + boolean setChannel = channelRef.compareAndSet(null, newChannel); + checkState(setChannel); + LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), + connectionAttempt); + } else { + Throwable cause = future.getCause(); + reconnectAgain(cause); + if (newChannel != null) { + newChannel.close(); + } } - else { - LOG.debug("failed to connect to {} [attempt {}]", address.toString(), connectionAttempt); - if (current != null) { - current.close(); + } else { + close(); + throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " + + connectionAttempts + " failed attempts"); } + } catch (Throwable e) { + LOG.error("Uncaught throwable", e); + throw Throwables.propagate(e); } - return channel; + } } -} } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java index 63c861ace74..169940f06a0 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java @@ -133,4 +133,8 @@ private void writeTaskMessage(ChannelBufferOutputStream bout, TaskMessage messag if (payload_len >0) bout.write(message.message()); } + + public ArrayList getMsgs() { + return msgs; + } } \ No newline at end of file From 91b8eb3840432e47b79f40abebec8304627732a8 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Thu, 28 May 2015 20:46:17 +0100 Subject: [PATCH 27/89] Bump version --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 338e54886ec..33220bbae42 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index 62ce62d018d..158e03a6cab 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 3251bfac8a4..49507167d6b 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 9ba7e5e31d2..4007ff9c0db 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 ../../pom.xml diff --git a/pom.xml b/pom.xml index a94193157f2..71fb460bc91 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.4 + 0.9.4-STORM-839 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.4 + v0.9.4-STORM-839 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 73a93cedf96..8494ad46e58 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index a5f99a88785..808594c6bb8 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 79f098165cf..2d67b4f99f9 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 3cee9242954..ff4e643416e 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4 + 0.9.4-STORM-839 ../../pom.xml org.apache.storm From 2c6b97133103ec6eb919219b0422a255e832c76d Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 28 May 2015 17:00:39 -0400 Subject: [PATCH 28/89] apply fix for STORM-790 --- storm-core/src/clj/backtype/storm/daemon/worker.clj | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj index d4c946745a9..d5569a77b29 100644 --- a/storm-core/src/clj/backtype/storm/daemon/worker.clj +++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj @@ -126,8 +126,10 @@ (when (not (.get remoteMap node+port)) (.put remoteMap node+port (ArrayList.))) (let [remote (.get remoteMap node+port)] - (.add remote (TaskMessage. task (.serialize serializer tuple))) - )))) + (if (not-nil? task) + (.add remote (TaskMessage. task (.serialize serializer tuple))) + (log-warn "Can't transfer tuple - task value is nil. tuple type: " (pr-str (type tuple)) " and information: " (pr-str tuple))) + )))) (local-transfer local) (disruptor/publish transfer-queue remoteMap) From 0eb514dcd8a5804e7155aae377e479d323a92ec6 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 28 May 2015 17:01:44 -0400 Subject: [PATCH 29/89] add STORM-790 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index f86558ddecf..6b72a34126b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.5 + * STORM-790: Log "task is null" instead of let worker died when task is null in transfer-fn * STORM-796: Add support for "error" command in ShellSpout * STORM-745: fix storm.cmd to evaluate 'shift' correctly with 'storm jar' From b7d84bdc7fd3de34f45a94131cdbb6bfbd3763dc Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Thu, 28 May 2015 22:27:31 +0100 Subject: [PATCH 30/89] Remove background flushing because it doesn't seem necessary. Netty's Channel queues up written data on an unbounded buffer. The background flushing seems to have been added to avoid this, but in practice it was probably doing it anyways because flushMessages(), which is called by send() doesn't check for isWritable. Moreover, queuing on an unbounded buffer seems fine because back pressure is provided by MAX_PENDING_TUPLE. If OOME occurs due to this buffer overflowing, it seems reasonable that one has to reduce MAX_PENDING_TUPLE, rather than Storm trying to cope with it by dropping messages. --- .../storm/messaging/netty/Client.java | 31 +------------------ 1 file changed, 1 insertion(+), 30 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 4e970356dd9..4bbe98983a8 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -128,8 +128,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private final ListeningScheduledExecutorService scheduler; protected final Map stormConf; - private AtomicReference pendingMessageBatch; - @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { closing = false; @@ -145,8 +143,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { int maxWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS)); retryPolicy = new StormBoundedExponentialBackoffRetry(minWaitMs, maxWaitMs, maxReconnectionAttempts); - pendingMessageBatch = new AtomicReference(new MessageBatch(messageBatchSize)); - // Initiate connection to remote destination bootstrap = createClientBootstrap(factory, bufferSize); dstAddress = new InetSocketAddress(host, port); @@ -155,8 +151,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { // Launch background flushing thread long initialDelayMs = Math.min(MINIMUM_INITIAL_DELAY_MS, maxWaitMs * maxReconnectionAttempts); - scheduler.scheduleWithFixedDelay(createBackgroundFlusher(), initialDelayMs, flushCheckIntervalMs, - TimeUnit.MILLISECONDS); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -175,28 +169,6 @@ private String prefixedName(InetSocketAddress dstAddress) { return ""; } - private Runnable createBackgroundFlusher() { - return new Runnable() { - @Override - public void run() { - if (!closing) { - LOG.debug("flushing pending messages to {} in background", dstAddressPrefixedName); - flushPendingMessages(); - } - } - }; - } - - private void flushPendingMessages() { - Channel channel = channelRef.get(); - if (connectionEstablished(channel)) { - MessageBatch toFlush = pendingMessageBatch.getAndSet(new MessageBatch(messageBatchSize)); - flushMessages(channel, toFlush); - } else { - closeChannelAndReconnect(channel); - } - } - /** * We will retry connection with exponential back-off policy */ @@ -276,7 +248,7 @@ public void send(Iterator msgs) { return; } - MessageBatch toSend = pendingMessageBatch.getAndSet(new MessageBatch(messageBatchSize)); + MessageBatch toSend = new MessageBatch(messageBatchSize); // Collect messages into batches (to optimize network throughput), then flush them. while (msgs.hasNext()) { @@ -381,7 +353,6 @@ public void close() { LOG.info("closing Netty Client {}", dstAddressPrefixedName); // Set closing to true to prevent any further reconnection attempts. closing = true; - flushPendingMessages(); closeChannel(); } } From 679e42bc1e38f51c2759667b03cb45322c6a793b Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Thu, 28 May 2015 22:31:35 +0100 Subject: [PATCH 31/89] Change to a SNAPSHOT version for deployment purposes --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 +- .../maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/dependency-reduced-pom.xml | 438 ++++++++++++++++++ storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 10 files changed, 448 insertions(+), 10 deletions(-) create mode 100644 storm-core/dependency-reduced-pom.xml diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 33220bbae42..dd4055d023a 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index 158e03a6cab..0bffdbeb465 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 49507167d6b..997847c60cd 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 4007ff9c0db..902396b1bf0 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index 71fb460bc91..10b5a13d954 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.4-STORM-839 + v0.9.4-STORM-839-SNAPSHOT https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 8494ad46e58..fe80f6bf4a5 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT ../../pom.xml diff --git a/storm-core/dependency-reduced-pom.xml b/storm-core/dependency-reduced-pom.xml new file mode 100644 index 00000000000..d971fdc3f66 --- /dev/null +++ b/storm-core/dependency-reduced-pom.xml @@ -0,0 +1,438 @@ + + + + storm + org.apache.storm + 0.9.4-STORM-839-SNAPSHOT + + 4.0.0 + org.apache.storm + storm-core + Storm Core + Storm Core Java API and Clojure implementation. + + src/jvm + test/jvm + + + ../conf + + + META-INF + ../ + + NOTICE + + + + + + src/dev + + + test/resources + + + + + com.theoryinpractise + clojure-maven-plugin + true + + + compile-clojure + compile + + compile + + + + test-clojure + test + + test-with-junit + + + ${test.extra.args} + + + + + + src/clj + + + test/clj + + false + true + + none + + + + + maven-surefire-report-plugin + + + ${project.build.directory}/test-reports + + + + + maven-shade-plugin + 2.2 + + + package + + shade + + + + + + org.apache.storm + maven-shade-clojure-transformer + ${project.version} + + + + false + true + true + false + + + org.apache.thrift:* + io.netty:netty + com.google.guava:guava + org.apache.httpcomponents:http* + org.apache.zookeeper:zookeeper + org.apache.curator:* + + + + + org.apache.thrift + org.apache.thrift7 + + + org.jboss.netty + org.apache.storm.netty + + + com.google.common + org.apache.storm.guava + + + com.google.thirdparty + org.apache.storm.guava.thirdparty + + + org.apache.http + org.apache.storm.http + + + org.apache.zookeeper + org.apache.storm.zookeeper + + + org.apache.curator + org.apache.storm.curator + + + + + + + + org.apache.thrift:* + + META-INF/LICENSE.txt + META-INF/NOTICE.txt + + + + io.netty:netty + + META-INF/LICENSE.txt + META-INF/NOTICE.txt + + + + commons-httpclient:commons-httpclient + + META-INF/LICENSE.txt + META-INF/NOTICE.txt + META-INF/README.txt + + + + org.apache.zookeeper:zookeeper + + LICENSE.txt + + + + + + + + + + org.clojure + clojure + 1.5.1 + compile + + + clj-time + clj-time + 0.4.1 + compile + + + joda-time + joda-time + 2.0 + compile + + + compojure + compojure + 1.1.3 + compile + + + org.clojure + core.incubator + 0.1.0 + compile + + + org.clojure + tools.macro + 0.1.0 + compile + + + clout + clout + 1.0.1 + compile + + + ring + ring-core + 1.1.5 + compile + + + commons-fileupload + commons-fileupload + 1.2.1 + compile + + + javax.servlet + servlet-api + 2.5 + compile + + + hiccup + hiccup + 0.3.6 + compile + + + ring + ring-devel + 0.3.11 + compile + + + clj-stacktrace + clj-stacktrace + 0.2.2 + compile + + + ring + ring-jetty-adapter + 0.3.11 + compile + + + ring + ring-servlet + 0.3.11 + compile + + + org.mortbay.jetty + jetty + 6.1.26 + compile + + + servlet-api + org.mortbay.jetty + + + + + org.mortbay.jetty + jetty-util + 6.1.26 + compile + + + org.clojure + tools.logging + 0.2.3 + compile + + + org.clojure + math.numeric-tower + 0.0.1 + compile + + + org.clojure + tools.cli + 0.2.4 + compile + + + commons-io + commons-io + 2.4 + compile + + + org.apache.commons + commons-exec + 1.1 + compile + + + commons-lang + commons-lang + 2.5 + compile + + + com.googlecode.json-simple + json-simple + 1.1 + compile + + + com.twitter + carbonite + 1.4.0 + compile + + + com.esotericsoftware.kryo + kryo + 2.21 + compile + + + com.esotericsoftware.reflectasm + reflectasm + 1.07 + shaded + compile + + + org.ow2.asm + asm + 4.0 + compile + + + com.esotericsoftware.minlog + minlog + 1.2 + compile + + + org.objenesis + objenesis + 1.2 + compile + + + com.twitter + chill-java + 0.3.5 + compile + + + org.yaml + snakeyaml + 1.11 + compile + + + commons-logging + commons-logging + 1.1.3 + compile + + + commons-codec + commons-codec + 1.6 + compile + + + com.googlecode.disruptor + disruptor + 2.10.1 + compile + + + org.jgrapht + jgrapht-core + 0.9.0 + compile + + + ch.qos.logback + logback-classic + 1.0.13 + compile + + + ch.qos.logback + logback-core + 1.0.13 + compile + + + org.slf4j + slf4j-api + 1.7.5 + compile + + + org.slf4j + log4j-over-slf4j + 1.6.6 + compile + + + jline + jline + 2.11 + compile + + + + diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 808594c6bb8..c919e4a719a 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 2d67b4f99f9..818dc0d3d41 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index ff4e643416e..ba438eeaaa5 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.4-STORM-839 + 0.9.4-STORM-839-SNAPSHOT ../../pom.xml org.apache.storm From 46b118a8d4f404d18babf706ec428131b558bd54 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 28 May 2015 17:40:22 -0400 Subject: [PATCH 32/89] apply missing parts of fix for STORM-130 --- CHANGELOG.md | 1 + storm-core/src/clj/backtype/storm/daemon/supervisor.clj | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6b72a34126b..8d242b7703f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-790: Log "task is null" instead of let worker died when task is null in transfer-fn * STORM-796: Add support for "error" command in ShellSpout * STORM-745: fix storm.cmd to evaluate 'shift' correctly with 'storm jar' + * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. ## 0.9.4 * STORM-559: ZkHosts in README should use 2181 as port. diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj index df7330a2627..9364d1fab73 100644 --- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj @@ -479,14 +479,15 @@ storm-id " from " master-code-dir) - (FileUtils/forceMkdir (File. tmproot)) (Utils/downloadFromMaster conf (master-stormjar-path master-code-dir) (supervisor-stormjar-path tmproot)) (Utils/downloadFromMaster conf (master-stormcode-path master-code-dir) (supervisor-stormcode-path tmproot)) (Utils/downloadFromMaster conf (master-stormconf-path master-code-dir) (supervisor-stormconf-path tmproot)) (extract-dir-from-jar (supervisor-stormjar-path tmproot) RESOURCES-SUBDIR tmproot) - (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) + (if-not (.exists (File. stormroot)) + (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) + (FileUtils/deleteDirectory (File. tmproot))) (log-message "Finished downloading code for storm id " storm-id " from " From 7dd1331faf9bf18d4576294cfe585bd84acced7e Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 28 May 2015 21:29:53 -0400 Subject: [PATCH 33/89] [maven-release-plugin] prepare release v0.9.5 --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 5aef022af6c..74ae50f4ffc 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index ca28c507f9a..bec08697c88 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index b44972308cd..6284865f58f 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 2a368bee888..01a262b749d 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/pom.xml b/pom.xml index 2b446c1b885..06e9f26fb5c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.5-SNAPSHOT + 0.9.5 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.5 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e1e13b07bc0..ebfe6ab7aa1 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index f40dedfd788..69db0c051a4 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 21d1a809a4a..83a1982c6c0 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index ea21d7c0c92..501fe8e49d3 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml org.apache.storm From 4dd99ccc63e54f76534f7571f9b941029b5caf44 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 28 May 2015 21:30:04 -0400 Subject: [PATCH 34/89] [maven-release-plugin] prepare for next development iteration --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 74ae50f4ffc..f2aacd046b4 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index bec08697c88..fe196881254 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 6284865f58f..a2654900785 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 01a262b749d..36f77c33df8 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index 06e9f26fb5c..80fa42b2794 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.5 + 0.9.6-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.5 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index ebfe6ab7aa1..dfca2a4ae23 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 69db0c051a4..3b7d17717c9 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 83a1982c6c0..bf6dcbf521e 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 501fe8e49d3..87b65a3860c 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm From 27a92e2aa3488c0203f500306e0583ff9e7e1e82 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Fri, 29 May 2015 10:32:16 +0100 Subject: [PATCH 35/89] Remove (now) dead comment and code --- storm-core/src/jvm/backtype/storm/Config.java | 5 ----- .../src/jvm/backtype/storm/messaging/netty/Client.java | 10 ---------- 2 files changed, 15 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 840701085c8..ab72c295120 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -91,11 +91,6 @@ public class Config extends HashMap { public static final Object STORM_NETTY_MESSAGE_BATCH_SIZE_SCHEMA = ConfigValidation.IntegerValidator; /** - * We check with this interval that whether the Netty channel is writable and try to write pending messages - */ - public static final String STORM_NETTY_FLUSH_CHECK_INTERVAL_MS = "storm.messaging.netty.flush.check.interval.ms"; - public static final Object STORM_NETTY_FLUSH_CHECK_INTERVAL_MS_SCHEMA = ConfigValidation.IntegerValidator; - /** * The delegate for serializing metadata, should be used for serialized objects stored in zookeeper and on disk. * This is NOT used for compressing serialized tuples sent between topologies. diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 4bbe98983a8..f2953fc44b9 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -60,16 +60,12 @@ * - Connecting and reconnecting are performed asynchronously. * - Note: The current implementation drops any messages that are being enqueued for sending if the connection to * the remote destination is currently unavailable. - * - A background flusher thread is run in the background. It will, at fixed intervals, check for any pending messages - * (i.e. messages buffered in memory) and flush them to the remote destination iff background flushing is currently - * enabled. */ public class Client extends ConnectionWithStatus implements IStatefulObject { private static final Logger LOG = LoggerFactory.getLogger(Client.class); private static final String PREFIX = "Netty-Client-"; private static final long NO_DELAY_MS = 0L; - private static final long MINIMUM_INITIAL_DELAY_MS = 30000L; private final StormBoundedExponentialBackoffRetry retryPolicy; private final ClientBootstrap bootstrap; @@ -126,17 +122,14 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private final int messageBatchSize; private final ListeningScheduledExecutorService scheduler; - protected final Map stormConf; @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { closing = false; - this.stormConf = stormConf; this.scheduler = MoreExecutors.listeningDecorator(scheduler); int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize); messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); - int flushCheckIntervalMs = Utils.getInt(stormConf.get(Config.STORM_NETTY_FLUSH_CHECK_INTERVAL_MS), 10); maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES)); int minWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); @@ -148,9 +141,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { dstAddress = new InetSocketAddress(host, port); dstAddressPrefixedName = prefixedName(dstAddress); scheduleConnect(NO_DELAY_MS); - - // Launch background flushing thread - long initialDelayMs = Math.min(MINIMUM_INITIAL_DELAY_MS, maxWaitMs * maxReconnectionAttempts); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { From fdb394c158ccd87c0a06c060239766d666a8db5a Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Fri, 29 May 2015 11:28:20 +0100 Subject: [PATCH 36/89] Accidentally committed generated file --- storm-core/dependency-reduced-pom.xml | 438 -------------------------- 1 file changed, 438 deletions(-) delete mode 100644 storm-core/dependency-reduced-pom.xml diff --git a/storm-core/dependency-reduced-pom.xml b/storm-core/dependency-reduced-pom.xml deleted file mode 100644 index d971fdc3f66..00000000000 --- a/storm-core/dependency-reduced-pom.xml +++ /dev/null @@ -1,438 +0,0 @@ - - - - storm - org.apache.storm - 0.9.4-STORM-839-SNAPSHOT - - 4.0.0 - org.apache.storm - storm-core - Storm Core - Storm Core Java API and Clojure implementation. - - src/jvm - test/jvm - - - ../conf - - - META-INF - ../ - - NOTICE - - - - - - src/dev - - - test/resources - - - - - com.theoryinpractise - clojure-maven-plugin - true - - - compile-clojure - compile - - compile - - - - test-clojure - test - - test-with-junit - - - ${test.extra.args} - - - - - - src/clj - - - test/clj - - false - true - - none - - - - - maven-surefire-report-plugin - - - ${project.build.directory}/test-reports - - - - - maven-shade-plugin - 2.2 - - - package - - shade - - - - - - org.apache.storm - maven-shade-clojure-transformer - ${project.version} - - - - false - true - true - false - - - org.apache.thrift:* - io.netty:netty - com.google.guava:guava - org.apache.httpcomponents:http* - org.apache.zookeeper:zookeeper - org.apache.curator:* - - - - - org.apache.thrift - org.apache.thrift7 - - - org.jboss.netty - org.apache.storm.netty - - - com.google.common - org.apache.storm.guava - - - com.google.thirdparty - org.apache.storm.guava.thirdparty - - - org.apache.http - org.apache.storm.http - - - org.apache.zookeeper - org.apache.storm.zookeeper - - - org.apache.curator - org.apache.storm.curator - - - - - - - - org.apache.thrift:* - - META-INF/LICENSE.txt - META-INF/NOTICE.txt - - - - io.netty:netty - - META-INF/LICENSE.txt - META-INF/NOTICE.txt - - - - commons-httpclient:commons-httpclient - - META-INF/LICENSE.txt - META-INF/NOTICE.txt - META-INF/README.txt - - - - org.apache.zookeeper:zookeeper - - LICENSE.txt - - - - - - - - - - org.clojure - clojure - 1.5.1 - compile - - - clj-time - clj-time - 0.4.1 - compile - - - joda-time - joda-time - 2.0 - compile - - - compojure - compojure - 1.1.3 - compile - - - org.clojure - core.incubator - 0.1.0 - compile - - - org.clojure - tools.macro - 0.1.0 - compile - - - clout - clout - 1.0.1 - compile - - - ring - ring-core - 1.1.5 - compile - - - commons-fileupload - commons-fileupload - 1.2.1 - compile - - - javax.servlet - servlet-api - 2.5 - compile - - - hiccup - hiccup - 0.3.6 - compile - - - ring - ring-devel - 0.3.11 - compile - - - clj-stacktrace - clj-stacktrace - 0.2.2 - compile - - - ring - ring-jetty-adapter - 0.3.11 - compile - - - ring - ring-servlet - 0.3.11 - compile - - - org.mortbay.jetty - jetty - 6.1.26 - compile - - - servlet-api - org.mortbay.jetty - - - - - org.mortbay.jetty - jetty-util - 6.1.26 - compile - - - org.clojure - tools.logging - 0.2.3 - compile - - - org.clojure - math.numeric-tower - 0.0.1 - compile - - - org.clojure - tools.cli - 0.2.4 - compile - - - commons-io - commons-io - 2.4 - compile - - - org.apache.commons - commons-exec - 1.1 - compile - - - commons-lang - commons-lang - 2.5 - compile - - - com.googlecode.json-simple - json-simple - 1.1 - compile - - - com.twitter - carbonite - 1.4.0 - compile - - - com.esotericsoftware.kryo - kryo - 2.21 - compile - - - com.esotericsoftware.reflectasm - reflectasm - 1.07 - shaded - compile - - - org.ow2.asm - asm - 4.0 - compile - - - com.esotericsoftware.minlog - minlog - 1.2 - compile - - - org.objenesis - objenesis - 1.2 - compile - - - com.twitter - chill-java - 0.3.5 - compile - - - org.yaml - snakeyaml - 1.11 - compile - - - commons-logging - commons-logging - 1.1.3 - compile - - - commons-codec - commons-codec - 1.6 - compile - - - com.googlecode.disruptor - disruptor - 2.10.1 - compile - - - org.jgrapht - jgrapht-core - 0.9.0 - compile - - - ch.qos.logback - logback-classic - 1.0.13 - compile - - - ch.qos.logback - logback-core - 1.0.13 - compile - - - org.slf4j - slf4j-api - 1.7.5 - compile - - - org.slf4j - log4j-over-slf4j - 1.6.6 - compile - - - jline - jline - 2.11 - compile - - - - From 36eff0a409336d9247ce2e96b52fcf9630a446fb Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Fri, 29 May 2015 11:30:38 +0100 Subject: [PATCH 37/89] Remove dead method --- .../src/jvm/backtype/storm/messaging/netty/MessageBatch.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java index 169940f06a0..63c861ace74 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java @@ -133,8 +133,4 @@ private void writeTaskMessage(ChannelBufferOutputStream bout, TaskMessage messag if (payload_len >0) bout.write(message.message()); } - - public ArrayList getMsgs() { - return msgs; - } } \ No newline at end of file From 884f496dcd06bc9413c04fdf730aca2cfb4239c6 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Fri, 29 May 2015 11:32:39 +0100 Subject: [PATCH 38/89] Remove comment line I forgot to remove --- storm-core/src/jvm/backtype/storm/Config.java | 1 - 1 file changed, 1 deletion(-) diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index ab72c295120..77e7cbc233a 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -90,7 +90,6 @@ public class Config extends HashMap { public static final String STORM_NETTY_MESSAGE_BATCH_SIZE = "storm.messaging.netty.transfer.batch.size"; public static final Object STORM_NETTY_MESSAGE_BATCH_SIZE_SCHEMA = ConfigValidation.IntegerValidator; - /** /** * The delegate for serializing metadata, should be used for serialized objects stored in zookeeper and on disk. * This is NOT used for compressing serialized tuples sent between topologies. From aa5c2d719bb3913285d4274cfcf8364df958b1ff Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Fri, 29 May 2015 14:40:47 +0100 Subject: [PATCH 39/89] Do not block in Connector. This task runs on a single (by default) thread that is shared among all Clients. If the task blocks, other reconnection requests can't be processed, resulting in a lot of messages being dropped. By not blocking, the thread should be able to service reconnection requests a lot quicker. --- .../storm/messaging/netty/Client.java | 39 +++++++++++-------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index f2953fc44b9..340d43b44aa 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -390,6 +390,8 @@ public String toString() { /** * Asynchronously establishes a Netty connection to the remote address + * This task runs on a single (by default) thread shared among all clients, and thus + * should not perform operations that block. */ private class Connector implements Runnable { @@ -399,7 +401,7 @@ public Connector(InetSocketAddress address) { this.address = address; } - private void reconnectAgain(Throwable t) { + private void reschedule(Throwable t) { String baseMsg = String.format("connection attempt %s to %s failed", connectionAttempts, dstAddressPrefixedName); String failureMsg = (t == null) ? baseMsg : baseMsg + ": " + t.toString(); @@ -413,26 +415,31 @@ private void reconnectAgain(Throwable t) { public void run() { try { if (reconnectingAllowed()) { - int connectionAttempt = connectionAttempts.getAndIncrement(); + final int connectionAttempt = connectionAttempts.getAndIncrement(); totalConnectionAttempts.getAndIncrement(); LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); ChannelFuture future = bootstrap.connect(address); - future.awaitUninterruptibly(); - Channel newChannel = future.getChannel(); - - if (future.isSuccess() && connectionEstablished(newChannel)) { - boolean setChannel = channelRef.compareAndSet(null, newChannel); - checkState(setChannel); - LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), - connectionAttempt); - } else { - Throwable cause = future.getCause(); - reconnectAgain(cause); - if (newChannel != null) { - newChannel.close(); + future.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + // This call returns immediately + Channel newChannel = future.getChannel(); + + if (future.isSuccess() && connectionEstablished(newChannel)) { + boolean setChannel = channelRef.compareAndSet(null, newChannel); + checkState(setChannel); + LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), + connectionAttempt); + } else { + Throwable cause = future.getCause(); + reschedule(cause); + if (newChannel != null) { + newChannel.close(); + } + } } - } + }); } else { close(); throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " + From 6453588ac64369e8c11ddb862e71372aad66147b Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 29 May 2015 10:37:49 -0400 Subject: [PATCH 40/89] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 4dd99ccc63e54f76534f7571f9b941029b5caf44. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index f2aacd046b4..74ae50f4ffc 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index fe196881254..bec08697c88 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index a2654900785..6284865f58f 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 36f77c33df8..01a262b749d 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/pom.xml b/pom.xml index 80fa42b2794..06e9f26fb5c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6-SNAPSHOT + 0.9.5 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.5 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index dfca2a4ae23..ebfe6ab7aa1 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 3b7d17717c9..69db0c051a4 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index bf6dcbf521e..83a1982c6c0 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 87b65a3860c..501fe8e49d3 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.5 ../../pom.xml org.apache.storm From ee7a7bd0986c7b3851e0954472e71df6e78245b8 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 29 May 2015 10:38:23 -0400 Subject: [PATCH 41/89] Revert "[maven-release-plugin] prepare release v0.9.5" This reverts commit 7dd1331faf9bf18d4576294cfe585bd84acced7e. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 74ae50f4ffc..5aef022af6c 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index bec08697c88..ca28c507f9a 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 6284865f58f..b44972308cd 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 01a262b749d..2a368bee888 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index 06e9f26fb5c..2b446c1b885 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.5 + 0.9.5-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.5 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index ebfe6ab7aa1..e1e13b07bc0 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 69db0c051a4..f40dedfd788 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 83a1982c6c0..21d1a809a4a 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 501fe8e49d3..ea21d7c0c92 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.5-SNAPSHOT ../../pom.xml org.apache.storm From c9c47b36071c02488ee1e75eb3851e58191b7534 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 29 May 2015 10:50:30 -0400 Subject: [PATCH 42/89] [maven-release-plugin] prepare release v0.9.5 --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 5aef022af6c..74ae50f4ffc 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index ca28c507f9a..bec08697c88 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index b44972308cd..6284865f58f 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 2a368bee888..01a262b749d 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/pom.xml b/pom.xml index 2b446c1b885..06e9f26fb5c 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.5-SNAPSHOT + 0.9.5 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.5 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e1e13b07bc0..ebfe6ab7aa1 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index f40dedfd788..69db0c051a4 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 21d1a809a4a..83a1982c6c0 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index ea21d7c0c92..501fe8e49d3 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5-SNAPSHOT + 0.9.5 ../../pom.xml org.apache.storm From f3824034287ceec3e6da92dbfb819c0eef11dfad Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Fri, 29 May 2015 10:50:38 -0400 Subject: [PATCH 43/89] [maven-release-plugin] prepare for next development iteration --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 74ae50f4ffc..f2aacd046b4 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index bec08697c88..fe196881254 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 6284865f58f..a2654900785 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 01a262b749d..36f77c33df8 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index 06e9f26fb5c..80fa42b2794 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.5 + 0.9.6-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.5 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index ebfe6ab7aa1..dfca2a4ae23 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 69db0c051a4..3b7d17717c9 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 83a1982c6c0..bf6dcbf521e 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 501fe8e49d3..87b65a3860c 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.5 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm From ad8112d10d662ae81498d11f78a602b97243a142 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Sun, 31 May 2015 00:54:31 +0100 Subject: [PATCH 44/89] Log error message for dropping messages only once per connection error (logging it everytime on send was flooding the log). --- .../storm/messaging/netty/Client.java | 24 +++++++++++++++---- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 340d43b44aa..187bba3fbd4 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -233,7 +233,11 @@ public void send(Iterator msgs) { Channel channel = channelRef.get(); if (!connectionEstablished(channel)) { // Closing the channel and reconnecting should be done before handling the messages. - closeChannelAndReconnect(channel); + boolean reconnectScheduled = closeChannelAndReconnect(channel); + if(reconnectScheduled){ + // Log the connection error only once + LOG.error("connection to {} is unavailable", dstAddressPrefixedName); + } handleMessagesWhenConnectionIsUnavailable(msgs); return; } @@ -267,7 +271,6 @@ private boolean hasMessages(Iterator msgs) { * succeed or not, and how long the recovery will take. */ private void handleMessagesWhenConnectionIsUnavailable(Iterator msgs) { - LOG.error("connection to {} is unavailable", dstAddressPrefixedName); dropMessages(msgs); } @@ -275,7 +278,6 @@ private void dropMessages(Iterator msgs) { // We consume the iterator by traversing and thus "emptying" it. int msgCount = iteratorSize(msgs); messagesLost.getAndAdd(msgCount); - LOG.error("dropping {} message(s) destined for {}", msgCount, dstAddressPrefixedName); } private int iteratorSize(Iterator msgs) { @@ -318,13 +320,21 @@ public void operationComplete(ChannelFuture future) throws Exception { }); } - private void closeChannelAndReconnect(Channel channel) { + /** + * Schedule a reconnect if we closed a non-null channel, and acquired the right to + * provide a replacement + * @param channel + * @return if the call scheduled a re-connect task + */ + private boolean closeChannelAndReconnect(Channel channel) { if (channel != null) { channel.close(); if (channelRef.compareAndSet(channel, null)) { scheduleConnect(NO_DELAY_MS); + return true; } } + return false; } private boolean containsMessages(MessageBatch batch) { @@ -431,6 +441,9 @@ public void operationComplete(ChannelFuture future) throws Exception { checkState(setChannel); LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), connectionAttempt); + if(messagesLost.get() > 0){ + LOG.warn("Re-connection to {} was successful but {} messages has been lost so far", address.toString(), messagesLost.get()); + } } else { Throwable cause = future.getCause(); reschedule(cause); @@ -443,7 +456,8 @@ public void operationComplete(ChannelFuture future) throws Exception { } else { close(); throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " + - connectionAttempts + " failed attempts"); + connectionAttempts + " failed attempts. " + messagesLost.get() + " messages were lost"); + } } catch (Throwable e) { LOG.error("Uncaught throwable", e); From ee4e94a01c29caacea480b12d57a7d77174bb9be Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Sun, 31 May 2015 00:54:52 +0100 Subject: [PATCH 45/89] Remove obsolete metrics --- .../src/jvm/backtype/storm/messaging/netty/Client.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 187bba3fbd4..ac36035142d 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -103,12 +103,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private final AtomicInteger messagesLost = new AtomicInteger(0); - /** - * Number of messages buffered in memory. - */ - private final AtomicLong pendingMessages = new AtomicLong(0); - - /** * This flag is set to true if and only if a client instance is being closed. */ @@ -371,7 +365,6 @@ public Object getState() { HashMap ret = new HashMap(); ret.put("reconnects", totalConnectionAttempts.getAndSet(0)); ret.put("sent", messagesSent.getAndSet(0)); - ret.put("pending", pendingMessages.get()); ret.put("lostOnSend", messagesLost.getAndSet(0)); ret.put("dest", dstAddress.toString()); String src = srcAddressName(); From 084c5a0b485620aa746d107419d403bf601b23d0 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Mon, 1 Jun 2015 16:07:03 +0100 Subject: [PATCH 46/89] Bring back pending message metric --- .../src/jvm/backtype/storm/messaging/netty/Client.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index ac36035142d..2c7f3dbd81c 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -103,6 +103,12 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private final AtomicInteger messagesLost = new AtomicInteger(0); + /** + * Number of messages buffered in memory. + */ + private final AtomicLong pendingMessages = new AtomicLong(0); + + /** * This flag is set to true if and only if a client instance is being closed. */ @@ -295,11 +301,14 @@ private void flushMessages(Channel channel, final MessageBatch batch) { return; } + final int numMessages = batch.size(); LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString()); + pendingMessages.addAndGet(numMessages); ChannelFuture future = channel.write(batch); future.addListener(new ChannelFutureListener() { public void operationComplete(ChannelFuture future) throws Exception { + pendingMessages.addAndGet(0 - numMessages); if (future.isSuccess()) { LOG.debug("sent {} messages to {}", numMessages, dstAddressPrefixedName); messagesSent.getAndAdd(batch.size()); @@ -365,6 +374,7 @@ public Object getState() { HashMap ret = new HashMap(); ret.put("reconnects", totalConnectionAttempts.getAndSet(0)); ret.put("sent", messagesSent.getAndSet(0)); + ret.put("pending", pendingMessages.get()); ret.put("lostOnSend", messagesLost.getAndSet(0)); ret.put("dest", dstAddress.toString()); String src = srcAddressName(); From 832b5db2ec5101601894d8484f0d1707d98331e6 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Tue, 2 Jun 2015 23:25:13 +0100 Subject: [PATCH 47/89] Bring back the batching of in-between calls to send. --- .../storm/messaging/netty/Client.java | 294 ++++++++++++------ .../storm/messaging/netty/Context.java | 16 +- .../storm/messaging/netty/MessageBatch.java | 4 + 3 files changed, 214 insertions(+), 100 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 2c7f3dbd81c..0d754485a38 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -23,14 +23,14 @@ import backtype.storm.metric.api.IStatefulObject; import backtype.storm.utils.StormBoundedExponentialBackoffRetry; import backtype.storm.utils.Utils; -import com.google.common.base.Throwables; -import com.google.common.util.concurrent.ListeningScheduledExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import org.jboss.netty.bootstrap.ClientBootstrap; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; import org.jboss.netty.channel.ChannelFuture; import org.jboss.netty.channel.ChannelFutureListener; +import org.jboss.netty.util.HashedWheelTimer; +import org.jboss.netty.util.Timeout; +import org.jboss.netty.util.TimerTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +41,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -121,12 +120,16 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private final int messageBatchSize; - private final ListeningScheduledExecutorService scheduler; + private final HashedWheelTimer scheduler; + + private final Object pendingMessageLock = new Object(); + private MessageBatch pendingMessage; + private Timeout pendingFlush; @SuppressWarnings("rawtypes") - Client(Map stormConf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { + Client(Map stormConf, ChannelFactory factory, HashedWheelTimer scheduler, String host, int port) { closing = false; - this.scheduler = MoreExecutors.listeningDecorator(scheduler); + this.scheduler = scheduler; int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize); messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); @@ -141,6 +144,10 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { dstAddress = new InetSocketAddress(host, port); dstAddressPrefixedName = prefixedName(dstAddress); scheduleConnect(NO_DELAY_MS); + + // Dummy values to avoid null checks + pendingMessage = new MessageBatch(messageBatchSize); + pendingFlush = scheduler.newTimeout(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -163,7 +170,7 @@ private String prefixedName(InetSocketAddress dstAddress) { * We will retry connection with exponential back-off policy */ private void scheduleConnect(long delayMs) { - scheduler.schedule(new Connector(dstAddress), delayMs, TimeUnit.MILLISECONDS); + scheduler.newTimeout(new Connect(dstAddress), delayMs, TimeUnit.MILLISECONDS); } private boolean reconnectingAllowed() { @@ -230,49 +237,122 @@ public void send(Iterator msgs) { return; } - Channel channel = channelRef.get(); - if (!connectionEstablished(channel)) { - // Closing the channel and reconnecting should be done before handling the messages. - boolean reconnectScheduled = closeChannelAndReconnect(channel); - if(reconnectScheduled){ - // Log the connection error only once - LOG.error("connection to {} is unavailable", dstAddressPrefixedName); - } - handleMessagesWhenConnectionIsUnavailable(msgs); + Channel channel = getConnectedChannel(); + if (channel == null) { + /* + * Connection is unavailable. We will drop pending messages and let at-least-once message replay kick in. + * + * Another option would be to buffer the messages in memory. But this option has the risk of causing OOM errors, + * especially for topologies that disable message acking because we don't know whether the connection recovery will + * succeed or not, and how long the recovery will take. + */ + dropMessages(msgs); return; } - MessageBatch toSend = new MessageBatch(messageBatchSize); + MessageBatch replacement = new MessageBatch(messageBatchSize); + MessageBatch previous; + synchronized (pendingMessageLock) { + // pendingMessage is never null + previous = pendingMessage; + pendingMessage = replacement; + + // We are flushing the pending messages, therefore we can cancel the current pending flush + // The cancel is idempotent + pendingFlush.cancel(); + } + + // Collect messages into batches (to optimize network throughput) + Batches batches = createBatches(previous, msgs); + + // Then flush the batches that are full + flushMessages(channel, batches.fullBatches); + + if (batches.unfilled.isEmpty()) { + // All messages ended up neatly into batches; there are no unfilled MessageBatch + return; + } + + if (channel.isWritable()) { + // Netty's internal buffer is not full. We should write the unfilled MessageBatch immediately + // to reduce latency + flushMessages(channel, batches.unfilled); + } else { + // We have an unfilled MessageBatch, but Netty's internal buffer is full, meaning that we have time. + // In this situation, waiting for more messages before handing it to Netty yields better throughput + queueUp(channel, batches.unfilled); + } + } + + private void queueUp(Channel channel, MessageBatch unfilled) { + Batches batches; + synchronized (pendingMessageLock) { + batches = createBatches(pendingMessage, unfilled.getMsgs().iterator()); + pendingMessage = batches.unfilled; + + if(!pendingMessage.isEmpty()) { + // We have a MessageBatch that isn't full yet, so we will wait for more messages. + // However, we don't want to wait indefinitely so we schedule a timeout which flushes + // this batch if it's still not flushed after a delay - // Collect messages into batches (to optimize network throughput), then flush them. + // First, cancel the currently pending flush, because we just saw that Netty's + // buffer is full and thus we know we can wait longer + pendingFlush.cancel(); + + // Schedule the new flush + pendingFlush = scheduler.newTimeout(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); + } + } + + // MessageBatches that were filled are immediately handed to Netty + flushMessages(channel, batches.fullBatches); + + } + + + private static class Batches { + final List fullBatches; + final MessageBatch unfilled; + + private Batches(List fullBatches, MessageBatch unfilled) { + this.fullBatches = fullBatches; + this.unfilled = unfilled; + } + } + + private Batches createBatches(MessageBatch previous, Iterator msgs){ + List ret = new ArrayList(); while (msgs.hasNext()) { TaskMessage message = msgs.next(); - toSend.add(message); - if (toSend.isFull()) { - flushMessages(channel, toSend); - toSend = new MessageBatch(messageBatchSize); + previous.add(message); + if (previous.isFull()) { + ret.add(previous); + previous = new MessageBatch(messageBatchSize); } } - // Handle any remaining messages in case the "last" batch was not full. - flushMessages(channel, toSend); + return new Batches(ret, previous); + } + private Channel getConnectedChannel() { + Channel channel = channelRef.get(); + if (connectionEstablished(channel)) { + return channel; + } else { + // Closing the channel and reconnecting should be done before handling the messages. + boolean reconnectScheduled = closeChannelAndReconnect(channel); + if (reconnectScheduled) { + // Log the connection error only once + LOG.error("connection to {} is unavailable", dstAddressPrefixedName); + } + return null; + } } private boolean hasMessages(Iterator msgs) { return msgs != null && msgs.hasNext(); } - /** - * We will drop pending messages and let at-least-once message replay kick in. - * - * Another option would be to buffer the messages in memory. But this option has the risk of causing OOM errors, - * especially for topologies that disable message acking because we don't know whether the connection recovery will - * succeed or not, and how long the recovery will take. - */ - private void handleMessagesWhenConnectionIsUnavailable(Iterator msgs) { - dropMessages(msgs); - } private void dropMessages(Iterator msgs) { // We consume the iterator by traversing and thus "emptying" it. @@ -280,6 +360,10 @@ private void dropMessages(Iterator msgs) { messagesLost.getAndAdd(msgCount); } + private void dropMessages(MessageBatch msgs) { + messagesLost.getAndAdd(msgs.size()); + } + private int iteratorSize(Iterator msgs) { int size = 0; if (msgs != null) { @@ -291,20 +375,23 @@ private int iteratorSize(Iterator msgs) { return size; } + private void flushMessages(Channel channel, List batches) { + for (MessageBatch batch : batches) { + flushMessages(channel, batch); + } + } + + /** * Asynchronously writes the message batch to the channel. * * If the write operation fails, then we will close the channel and trigger a reconnect. */ private void flushMessages(Channel channel, final MessageBatch batch) { - if (!containsMessages(batch)) { - return; - } - - final int numMessages = batch.size(); LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString()); pendingMessages.addAndGet(numMessages); + ChannelFuture future = channel.write(batch); future.addListener(new ChannelFutureListener() { public void operationComplete(ChannelFuture future) throws Exception { @@ -325,7 +412,7 @@ public void operationComplete(ChannelFuture future) throws Exception { /** * Schedule a reconnect if we closed a non-null channel, and acquired the right to - * provide a replacement + * provide a replacement by successfully setting a null to the channel field * @param channel * @return if the call scheduled a re-connect task */ @@ -340,15 +427,8 @@ private boolean closeChannelAndReconnect(Channel channel) { return false; } - private boolean containsMessages(MessageBatch batch) { - return batch != null && !batch.isEmpty(); - } - /** * Gracefully close this client. - * - * We will attempt to send any pending messages (i.e. messages currently buffered in memory) before closing the - * client. */ @Override public void close() { @@ -356,10 +436,12 @@ public void close() { LOG.info("closing Netty Client {}", dstAddressPrefixedName); // Set closing to true to prevent any further reconnection attempts. closing = true; + closeChannel(); } } + private void closeChannel() { Channel channel = channelRef.get(); if (channel != null) { @@ -401,16 +483,57 @@ public String toString() { return String.format("Netty client for connecting to %s", dstAddressPrefixedName); } + /** + * Asynchronously flushes pending messages to the remote address, if they have not been + * flushed by other means. + * This task runs on a single thread shared among all clients, and thus + * should not perform operations that block or are expensive. + */ + private class Flush implements TimerTask { + private final MessageBatch instructor; + + private Flush(MessageBatch instructor) { + this.instructor = instructor; + } + + @Override + public void run(Timeout timeout) throws Exception { + MessageBatch toSend; + MessageBatch replacement = new MessageBatch(messageBatchSize); + synchronized (pendingMessageLock){ + if(instructor == pendingMessage){ + // It's still the batch which scheduled this timeout + toSend = pendingMessage; + pendingMessage = replacement; + checkState(!toSend.isFull(), "Only unfilled batches should get timeouts scheduled"); + } else { + // It's no longer the batch which scheduled this timeout + // No need to work on this one + toSend = null; + } + } + + if(toSend!=null){ + Channel channel = getConnectedChannel(); + if(channel == null) { + dropMessages(toSend); + } else { + flushMessages(channel, toSend); + } + } + } + } + /** * Asynchronously establishes a Netty connection to the remote address - * This task runs on a single (by default) thread shared among all clients, and thus + * This task runs on a single thread shared among all clients, and thus * should not perform operations that block. */ - private class Connector implements Runnable { + private class Connect implements TimerTask { private final InetSocketAddress address; - public Connector(InetSocketAddress address) { + public Connect(InetSocketAddress address) { this.address = address; } @@ -425,46 +548,41 @@ private void reschedule(Throwable t) { @Override - public void run() { - try { - if (reconnectingAllowed()) { - final int connectionAttempt = connectionAttempts.getAndIncrement(); - totalConnectionAttempts.getAndIncrement(); - - LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); - ChannelFuture future = bootstrap.connect(address); - future.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - // This call returns immediately - Channel newChannel = future.getChannel(); - - if (future.isSuccess() && connectionEstablished(newChannel)) { - boolean setChannel = channelRef.compareAndSet(null, newChannel); - checkState(setChannel); - LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), - connectionAttempt); - if(messagesLost.get() > 0){ - LOG.warn("Re-connection to {} was successful but {} messages has been lost so far", address.toString(), messagesLost.get()); - } - } else { - Throwable cause = future.getCause(); - reschedule(cause); - if (newChannel != null) { - newChannel.close(); - } + public void run(Timeout timeout) throws Exception { + if (reconnectingAllowed()) { + final int connectionAttempt = connectionAttempts.getAndIncrement(); + totalConnectionAttempts.getAndIncrement(); + + LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); + ChannelFuture future = bootstrap.connect(address); + future.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + // This call returns immediately + Channel newChannel = future.getChannel(); + + if (future.isSuccess() && connectionEstablished(newChannel)) { + boolean setChannel = channelRef.compareAndSet(null, newChannel); + checkState(setChannel); + LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), + connectionAttempt); + if (messagesLost.get() > 0) { + LOG.warn("Re-connection to {} was successful but {} messages has been lost so far", address.toString(), messagesLost.get()); + } + } else { + Throwable cause = future.getCause(); + reschedule(cause); + if (newChannel != null) { + newChannel.close(); } } - }); - } else { - close(); - throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " + - connectionAttempts + " failed attempts. " + messagesLost.get() + " messages were lost"); + } + }); + } else { + close(); + throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " + + connectionAttempts + " failed attempts. " + messagesLost.get() + " messages were lost"); - } - } catch (Throwable e) { - LOG.error("Uncaught throwable", e); - throw Throwables.propagate(e); } } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java index f592affd3cf..64f67ba57fc 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java @@ -18,6 +18,7 @@ package backtype.storm.messaging.netty; import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; +import org.jboss.netty.util.HashedWheelTimer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.concurrent.Executors; @@ -40,8 +41,7 @@ public class Context implements IContext { private volatile Vector connections; private NioClientSocketChannelFactory clientChannelFactory; - private ScheduledExecutorService clientScheduleService; - private final int MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE = 10; + private HashedWheelTimer clientScheduleService; /** * initialization per Storm configuration @@ -63,9 +63,7 @@ public void prepare(Map storm_conf) { Executors.newCachedThreadPool(workerFactory)); } - int otherWorkers = Utils.getInt(storm_conf.get(Config.TOPOLOGY_WORKERS), 1) - 1; - int poolSize = Math.min(Math.max(1, otherWorkers), MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE); - clientScheduleService = Executors.newScheduledThreadPool(poolSize, new NettyRenameThreadFactory("client-schedule-service")); + clientScheduleService = new HashedWheelTimer(new NettyRenameThreadFactory("client-schedule-timer"), 10, TimeUnit.MILLISECONDS); } /** @@ -91,18 +89,12 @@ public IConnection connect(String storm_id, String host, int port) { * terminate this context */ public void term() { - clientScheduleService.shutdown(); + clientScheduleService.stop(); for (IConnection conn : connections) { conn.close(); } - try { - clientScheduleService.awaitTermination(30, TimeUnit.SECONDS); - } catch (InterruptedException e) { - LOG.error("Error when shutting down client scheduler", e); - } - connections = null; //we need to release resources associated with client channel factory diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java index 63c861ace74..169940f06a0 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java @@ -133,4 +133,8 @@ private void writeTaskMessage(ChannelBufferOutputStream bout, TaskMessage messag if (payload_len >0) bout.write(message.message()); } + + public ArrayList getMsgs() { + return msgs; + } } \ No newline at end of file From afa638cdbf2440d88af3775c0f02b4dc792922b0 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Wed, 3 Jun 2015 00:26:53 +0100 Subject: [PATCH 48/89] Abandone the idea of scheduling timeouts as needed as it's still performing worse than the original version --- .../storm/messaging/netty/Client.java | 127 +++++++++--------- .../storm/messaging/netty/Context.java | 32 +++-- 2 files changed, 85 insertions(+), 74 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 0d754485a38..365288615d2 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -23,6 +23,7 @@ import backtype.storm.metric.api.IStatefulObject; import backtype.storm.utils.StormBoundedExponentialBackoffRetry; import backtype.storm.utils.Utils; +import com.google.common.base.Throwables; import org.jboss.netty.bootstrap.ClientBootstrap; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; @@ -41,6 +42,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -120,14 +122,14 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private final int messageBatchSize; - private final HashedWheelTimer scheduler; + private final ScheduledExecutorService scheduler; private final Object pendingMessageLock = new Object(); private MessageBatch pendingMessage; private Timeout pendingFlush; @SuppressWarnings("rawtypes") - Client(Map stormConf, ChannelFactory factory, HashedWheelTimer scheduler, String host, int port) { + Client(Map stormConf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { closing = false; this.scheduler = scheduler; int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); @@ -147,7 +149,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { // Dummy values to avoid null checks pendingMessage = new MessageBatch(messageBatchSize); - pendingFlush = scheduler.newTimeout(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); + scheduler.scheduleWithFixedDelay(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -489,37 +491,33 @@ public String toString() { * This task runs on a single thread shared among all clients, and thus * should not perform operations that block or are expensive. */ - private class Flush implements TimerTask { - private final MessageBatch instructor; - - private Flush(MessageBatch instructor) { - this.instructor = instructor; - } - + private class Flush implements Runnable { @Override - public void run(Timeout timeout) throws Exception { - MessageBatch toSend; - MessageBatch replacement = new MessageBatch(messageBatchSize); - synchronized (pendingMessageLock){ - if(instructor == pendingMessage){ - // It's still the batch which scheduled this timeout - toSend = pendingMessage; - pendingMessage = replacement; - checkState(!toSend.isFull(), "Only unfilled batches should get timeouts scheduled"); - } else { - // It's no longer the batch which scheduled this timeout - // No need to work on this one - toSend = null; - } - } - - if(toSend!=null){ + public void run() { + try { Channel channel = getConnectedChannel(); - if(channel == null) { - dropMessages(toSend); + if (channel == null || !channel.isWritable()) { + // Connection not available or buffer is full, no point in flushing + return; } else { + // Connection is available and there is room in Netty's buffer + MessageBatch toSend; + synchronized (pendingMessageLock) { + if(pendingMessage.isEmpty()){ + // Nothing to flush + return; + } else { + toSend = pendingMessage; + pendingMessage = new MessageBatch(messageBatchSize); + } + } + checkState(!toSend.isFull(), "Filled batches should never be in pendingMessage field"); + flushMessages(channel, toSend); } + }catch (Throwable e){ + LOG.error("Uncaught throwable", e); + throw Throwables.propagate(e); } } } @@ -529,7 +527,7 @@ public void run(Timeout timeout) throws Exception { * This task runs on a single thread shared among all clients, and thus * should not perform operations that block. */ - private class Connect implements TimerTask { + private class Connect implements Runnable { private final InetSocketAddress address; @@ -548,41 +546,46 @@ private void reschedule(Throwable t) { @Override - public void run(Timeout timeout) throws Exception { - if (reconnectingAllowed()) { - final int connectionAttempt = connectionAttempts.getAndIncrement(); - totalConnectionAttempts.getAndIncrement(); - - LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); - ChannelFuture future = bootstrap.connect(address); - future.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture future) throws Exception { - // This call returns immediately - Channel newChannel = future.getChannel(); - - if (future.isSuccess() && connectionEstablished(newChannel)) { - boolean setChannel = channelRef.compareAndSet(null, newChannel); - checkState(setChannel); - LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), - connectionAttempt); - if (messagesLost.get() > 0) { - LOG.warn("Re-connection to {} was successful but {} messages has been lost so far", address.toString(), messagesLost.get()); - } - } else { - Throwable cause = future.getCause(); - reschedule(cause); - if (newChannel != null) { - newChannel.close(); + public void run() { + try { + if (reconnectingAllowed()) { + final int connectionAttempt = connectionAttempts.getAndIncrement(); + totalConnectionAttempts.getAndIncrement(); + + LOG.debug("connecting to {} [attempt {}]", address.toString(), connectionAttempt); + ChannelFuture future = bootstrap.connect(address); + future.addListener(new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + // This call returns immediately + Channel newChannel = future.getChannel(); + + if (future.isSuccess() && connectionEstablished(newChannel)) { + boolean setChannel = channelRef.compareAndSet(null, newChannel); + checkState(setChannel); + LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), + connectionAttempt); + if (messagesLost.get() > 0) { + LOG.warn("Re-connection to {} was successful but {} messages has been lost so far", address.toString(), messagesLost.get()); + } + } else { + Throwable cause = future.getCause(); + reschedule(cause); + if (newChannel != null) { + newChannel.close(); + } } } - } - }); - } else { - close(); - throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " + - connectionAttempts + " failed attempts. " + messagesLost.get() + " messages were lost"); + }); + } else { + close(); + throw new RuntimeException("Giving up to scheduleConnect to " + dstAddressPrefixedName + " after " + + connectionAttempts + " failed attempts. " + messagesLost.get() + " messages were lost"); + } + }catch (Throwable e){ + LOG.error("Uncaught throwable", e); + throw Throwables.propagate(e); } } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java index 64f67ba57fc..7e0cb0da15d 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java @@ -18,7 +18,6 @@ package backtype.storm.messaging.netty; import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory; -import org.jboss.netty.util.HashedWheelTimer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.concurrent.Executors; @@ -35,13 +34,14 @@ public class Context implements IContext { private static final Logger LOG = LoggerFactory.getLogger(Context.class); - + @SuppressWarnings("rawtypes") private Map storm_conf; private volatile Vector connections; private NioClientSocketChannelFactory clientChannelFactory; - - private HashedWheelTimer clientScheduleService; + + private ScheduledExecutorService clientScheduleService; + private final int MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE = 10; /** * initialization per Storm configuration @@ -53,7 +53,7 @@ public void prepare(Map storm_conf) { //each context will have a single client channel factory int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS)); - ThreadFactory bossFactory = new NettyRenameThreadFactory("client" + "-boss"); + ThreadFactory bossFactory = new NettyRenameThreadFactory("client" + "-boss"); ThreadFactory workerFactory = new NettyRenameThreadFactory("client" + "-worker"); if (maxWorkers > 0) { clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), @@ -62,8 +62,10 @@ public void prepare(Map storm_conf) { clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), Executors.newCachedThreadPool(workerFactory)); } - - clientScheduleService = new HashedWheelTimer(new NettyRenameThreadFactory("client-schedule-timer"), 10, TimeUnit.MILLISECONDS); + + int otherWorkers = Utils.getInt(storm_conf.get(Config.TOPOLOGY_WORKERS), 1) - 1; + int poolSize = Math.min(Math.max(1, otherWorkers), MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE); + clientScheduleService = Executors.newScheduledThreadPool(poolSize, new NettyRenameThreadFactory("client-schedule-service")); } /** @@ -78,8 +80,8 @@ public IConnection bind(String storm_id, int port) { /** * establish a connection to a remote server */ - public IConnection connect(String storm_id, String host, int port) { - IConnection client = new Client(storm_conf, clientChannelFactory, + public IConnection connect(String storm_id, String host, int port) { + IConnection client = new Client(storm_conf, clientChannelFactory, clientScheduleService, host, port); connections.add(client); return client; @@ -89,12 +91,18 @@ public IConnection connect(String storm_id, String host, int port) { * terminate this context */ public void term() { - clientScheduleService.stop(); - + clientScheduleService.shutdown(); + for (IConnection conn : connections) { conn.close(); } - + + try { + clientScheduleService.awaitTermination(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.error("Error when shutting down client scheduler", e); + } + connections = null; //we need to release resources associated with client channel factory From 167be7e79896178021df4e407d79e7e8d72fba44 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Wed, 3 Jun 2015 00:30:25 +0100 Subject: [PATCH 49/89] Forgot to commit --- .../backtype/storm/messaging/netty/Client.java | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 365288615d2..5d9381e6533 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -149,7 +149,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { // Dummy values to avoid null checks pendingMessage = new MessageBatch(messageBatchSize); - scheduler.scheduleWithFixedDelay(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); + scheduler.scheduleWithFixedDelay(new Flush(), 10, 10, TimeUnit.MILLISECONDS); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -172,7 +172,7 @@ private String prefixedName(InetSocketAddress dstAddress) { * We will retry connection with exponential back-off policy */ private void scheduleConnect(long delayMs) { - scheduler.newTimeout(new Connect(dstAddress), delayMs, TimeUnit.MILLISECONDS); + scheduler.schedule(new Connect(dstAddress), delayMs, TimeUnit.MILLISECONDS); } private boolean reconnectingAllowed() { @@ -290,20 +290,8 @@ private void queueUp(Channel channel, MessageBatch unfilled) { Batches batches; synchronized (pendingMessageLock) { batches = createBatches(pendingMessage, unfilled.getMsgs().iterator()); + // We have a MessageBatch that isn't full yet, so we will wait for more messages. pendingMessage = batches.unfilled; - - if(!pendingMessage.isEmpty()) { - // We have a MessageBatch that isn't full yet, so we will wait for more messages. - // However, we don't want to wait indefinitely so we schedule a timeout which flushes - // this batch if it's still not flushed after a delay - - // First, cancel the currently pending flush, because we just saw that Netty's - // buffer is full and thus we know we can wait longer - pendingFlush.cancel(); - - // Schedule the new flush - pendingFlush = scheduler.newTimeout(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); - } } // MessageBatches that were filled are immediately handed to Netty From 4104a0a9006b57debb63f56c0dd21a44809aa380 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Wed, 3 Jun 2015 12:56:54 +0100 Subject: [PATCH 50/89] Revert the background flushing --- .../storm/messaging/netty/Client.java | 209 +++++------------- .../storm/messaging/netty/Context.java | 20 +- .../storm/messaging/netty/MessageBatch.java | 4 - 3 files changed, 60 insertions(+), 173 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 5d9381e6533..2c7f3dbd81c 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -24,14 +24,13 @@ import backtype.storm.utils.StormBoundedExponentialBackoffRetry; import backtype.storm.utils.Utils; import com.google.common.base.Throwables; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import org.jboss.netty.bootstrap.ClientBootstrap; import org.jboss.netty.channel.Channel; import org.jboss.netty.channel.ChannelFactory; import org.jboss.netty.channel.ChannelFuture; import org.jboss.netty.channel.ChannelFutureListener; -import org.jboss.netty.util.HashedWheelTimer; -import org.jboss.netty.util.Timeout; -import org.jboss.netty.util.TimerTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -122,16 +121,12 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private final int messageBatchSize; - private final ScheduledExecutorService scheduler; - - private final Object pendingMessageLock = new Object(); - private MessageBatch pendingMessage; - private Timeout pendingFlush; + private final ListeningScheduledExecutorService scheduler; @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, ScheduledExecutorService scheduler, String host, int port) { closing = false; - this.scheduler = scheduler; + this.scheduler = MoreExecutors.listeningDecorator(scheduler); int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize); messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); @@ -146,10 +141,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { dstAddress = new InetSocketAddress(host, port); dstAddressPrefixedName = prefixedName(dstAddress); scheduleConnect(NO_DELAY_MS); - - // Dummy values to avoid null checks - pendingMessage = new MessageBatch(messageBatchSize); - scheduler.scheduleWithFixedDelay(new Flush(), 10, 10, TimeUnit.MILLISECONDS); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -172,7 +163,7 @@ private String prefixedName(InetSocketAddress dstAddress) { * We will retry connection with exponential back-off policy */ private void scheduleConnect(long delayMs) { - scheduler.schedule(new Connect(dstAddress), delayMs, TimeUnit.MILLISECONDS); + scheduler.schedule(new Connector(dstAddress), delayMs, TimeUnit.MILLISECONDS); } private boolean reconnectingAllowed() { @@ -239,110 +230,49 @@ public void send(Iterator msgs) { return; } - Channel channel = getConnectedChannel(); - if (channel == null) { - /* - * Connection is unavailable. We will drop pending messages and let at-least-once message replay kick in. - * - * Another option would be to buffer the messages in memory. But this option has the risk of causing OOM errors, - * especially for topologies that disable message acking because we don't know whether the connection recovery will - * succeed or not, and how long the recovery will take. - */ - dropMessages(msgs); - return; - } - - MessageBatch replacement = new MessageBatch(messageBatchSize); - MessageBatch previous; - synchronized (pendingMessageLock) { - // pendingMessage is never null - previous = pendingMessage; - pendingMessage = replacement; - - // We are flushing the pending messages, therefore we can cancel the current pending flush - // The cancel is idempotent - pendingFlush.cancel(); - } - - // Collect messages into batches (to optimize network throughput) - Batches batches = createBatches(previous, msgs); - - // Then flush the batches that are full - flushMessages(channel, batches.fullBatches); - - if (batches.unfilled.isEmpty()) { - // All messages ended up neatly into batches; there are no unfilled MessageBatch + Channel channel = channelRef.get(); + if (!connectionEstablished(channel)) { + // Closing the channel and reconnecting should be done before handling the messages. + boolean reconnectScheduled = closeChannelAndReconnect(channel); + if(reconnectScheduled){ + // Log the connection error only once + LOG.error("connection to {} is unavailable", dstAddressPrefixedName); + } + handleMessagesWhenConnectionIsUnavailable(msgs); return; } - if (channel.isWritable()) { - // Netty's internal buffer is not full. We should write the unfilled MessageBatch immediately - // to reduce latency - flushMessages(channel, batches.unfilled); - } else { - // We have an unfilled MessageBatch, but Netty's internal buffer is full, meaning that we have time. - // In this situation, waiting for more messages before handing it to Netty yields better throughput - queueUp(channel, batches.unfilled); - } - } + MessageBatch toSend = new MessageBatch(messageBatchSize); - private void queueUp(Channel channel, MessageBatch unfilled) { - Batches batches; - synchronized (pendingMessageLock) { - batches = createBatches(pendingMessage, unfilled.getMsgs().iterator()); - // We have a MessageBatch that isn't full yet, so we will wait for more messages. - pendingMessage = batches.unfilled; - } - - // MessageBatches that were filled are immediately handed to Netty - flushMessages(channel, batches.fullBatches); - - } - - - private static class Batches { - final List fullBatches; - final MessageBatch unfilled; - - private Batches(List fullBatches, MessageBatch unfilled) { - this.fullBatches = fullBatches; - this.unfilled = unfilled; - } - } - - private Batches createBatches(MessageBatch previous, Iterator msgs){ - List ret = new ArrayList(); + // Collect messages into batches (to optimize network throughput), then flush them. while (msgs.hasNext()) { TaskMessage message = msgs.next(); - previous.add(message); - if (previous.isFull()) { - ret.add(previous); - previous = new MessageBatch(messageBatchSize); + toSend.add(message); + if (toSend.isFull()) { + flushMessages(channel, toSend); + toSend = new MessageBatch(messageBatchSize); } } - return new Batches(ret, previous); - } + // Handle any remaining messages in case the "last" batch was not full. + flushMessages(channel, toSend); - private Channel getConnectedChannel() { - Channel channel = channelRef.get(); - if (connectionEstablished(channel)) { - return channel; - } else { - // Closing the channel and reconnecting should be done before handling the messages. - boolean reconnectScheduled = closeChannelAndReconnect(channel); - if (reconnectScheduled) { - // Log the connection error only once - LOG.error("connection to {} is unavailable", dstAddressPrefixedName); - } - return null; - } } private boolean hasMessages(Iterator msgs) { return msgs != null && msgs.hasNext(); } + /** + * We will drop pending messages and let at-least-once message replay kick in. + * + * Another option would be to buffer the messages in memory. But this option has the risk of causing OOM errors, + * especially for topologies that disable message acking because we don't know whether the connection recovery will + * succeed or not, and how long the recovery will take. + */ + private void handleMessagesWhenConnectionIsUnavailable(Iterator msgs) { + dropMessages(msgs); + } private void dropMessages(Iterator msgs) { // We consume the iterator by traversing and thus "emptying" it. @@ -350,10 +280,6 @@ private void dropMessages(Iterator msgs) { messagesLost.getAndAdd(msgCount); } - private void dropMessages(MessageBatch msgs) { - messagesLost.getAndAdd(msgs.size()); - } - private int iteratorSize(Iterator msgs) { int size = 0; if (msgs != null) { @@ -365,23 +291,20 @@ private int iteratorSize(Iterator msgs) { return size; } - private void flushMessages(Channel channel, List batches) { - for (MessageBatch batch : batches) { - flushMessages(channel, batch); - } - } - - /** * Asynchronously writes the message batch to the channel. * * If the write operation fails, then we will close the channel and trigger a reconnect. */ private void flushMessages(Channel channel, final MessageBatch batch) { + if (!containsMessages(batch)) { + return; + } + + final int numMessages = batch.size(); LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString()); pendingMessages.addAndGet(numMessages); - ChannelFuture future = channel.write(batch); future.addListener(new ChannelFutureListener() { public void operationComplete(ChannelFuture future) throws Exception { @@ -402,7 +325,7 @@ public void operationComplete(ChannelFuture future) throws Exception { /** * Schedule a reconnect if we closed a non-null channel, and acquired the right to - * provide a replacement by successfully setting a null to the channel field + * provide a replacement * @param channel * @return if the call scheduled a re-connect task */ @@ -417,8 +340,15 @@ private boolean closeChannelAndReconnect(Channel channel) { return false; } + private boolean containsMessages(MessageBatch batch) { + return batch != null && !batch.isEmpty(); + } + /** * Gracefully close this client. + * + * We will attempt to send any pending messages (i.e. messages currently buffered in memory) before closing the + * client. */ @Override public void close() { @@ -426,12 +356,10 @@ public void close() { LOG.info("closing Netty Client {}", dstAddressPrefixedName); // Set closing to true to prevent any further reconnection attempts. closing = true; - closeChannel(); } } - private void closeChannel() { Channel channel = channelRef.get(); if (channel != null) { @@ -473,53 +401,16 @@ public String toString() { return String.format("Netty client for connecting to %s", dstAddressPrefixedName); } - /** - * Asynchronously flushes pending messages to the remote address, if they have not been - * flushed by other means. - * This task runs on a single thread shared among all clients, and thus - * should not perform operations that block or are expensive. - */ - private class Flush implements Runnable { - @Override - public void run() { - try { - Channel channel = getConnectedChannel(); - if (channel == null || !channel.isWritable()) { - // Connection not available or buffer is full, no point in flushing - return; - } else { - // Connection is available and there is room in Netty's buffer - MessageBatch toSend; - synchronized (pendingMessageLock) { - if(pendingMessage.isEmpty()){ - // Nothing to flush - return; - } else { - toSend = pendingMessage; - pendingMessage = new MessageBatch(messageBatchSize); - } - } - checkState(!toSend.isFull(), "Filled batches should never be in pendingMessage field"); - - flushMessages(channel, toSend); - } - }catch (Throwable e){ - LOG.error("Uncaught throwable", e); - throw Throwables.propagate(e); - } - } - } - /** * Asynchronously establishes a Netty connection to the remote address - * This task runs on a single thread shared among all clients, and thus + * This task runs on a single (by default) thread shared among all clients, and thus * should not perform operations that block. */ - private class Connect implements Runnable { + private class Connector implements Runnable { private final InetSocketAddress address; - public Connect(InetSocketAddress address) { + public Connector(InetSocketAddress address) { this.address = address; } @@ -553,7 +444,7 @@ public void operationComplete(ChannelFuture future) throws Exception { checkState(setChannel); LOG.debug("successfully connected to {}, {} [attempt {}]", address.toString(), newChannel.toString(), connectionAttempt); - if (messagesLost.get() > 0) { + if(messagesLost.get() > 0){ LOG.warn("Re-connection to {} was successful but {} messages has been lost so far", address.toString(), messagesLost.get()); } } else { @@ -571,7 +462,7 @@ public void operationComplete(ChannelFuture future) throws Exception { connectionAttempts + " failed attempts. " + messagesLost.get() + " messages were lost"); } - }catch (Throwable e){ + } catch (Throwable e) { LOG.error("Uncaught throwable", e); throw Throwables.propagate(e); } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java index 7e0cb0da15d..f592affd3cf 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java @@ -34,12 +34,12 @@ public class Context implements IContext { private static final Logger LOG = LoggerFactory.getLogger(Context.class); - + @SuppressWarnings("rawtypes") private Map storm_conf; private volatile Vector connections; private NioClientSocketChannelFactory clientChannelFactory; - + private ScheduledExecutorService clientScheduleService; private final int MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE = 10; @@ -53,7 +53,7 @@ public void prepare(Map storm_conf) { //each context will have a single client channel factory int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS)); - ThreadFactory bossFactory = new NettyRenameThreadFactory("client" + "-boss"); + ThreadFactory bossFactory = new NettyRenameThreadFactory("client" + "-boss"); ThreadFactory workerFactory = new NettyRenameThreadFactory("client" + "-worker"); if (maxWorkers > 0) { clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), @@ -62,7 +62,7 @@ public void prepare(Map storm_conf) { clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), Executors.newCachedThreadPool(workerFactory)); } - + int otherWorkers = Utils.getInt(storm_conf.get(Config.TOPOLOGY_WORKERS), 1) - 1; int poolSize = Math.min(Math.max(1, otherWorkers), MAX_CLIENT_SCHEDULER_THREAD_POOL_SIZE); clientScheduleService = Executors.newScheduledThreadPool(poolSize, new NettyRenameThreadFactory("client-schedule-service")); @@ -80,8 +80,8 @@ public IConnection bind(String storm_id, int port) { /** * establish a connection to a remote server */ - public IConnection connect(String storm_id, String host, int port) { - IConnection client = new Client(storm_conf, clientChannelFactory, + public IConnection connect(String storm_id, String host, int port) { + IConnection client = new Client(storm_conf, clientChannelFactory, clientScheduleService, host, port); connections.add(client); return client; @@ -91,18 +91,18 @@ public IConnection connect(String storm_id, String host, int port) { * terminate this context */ public void term() { - clientScheduleService.shutdown(); - + clientScheduleService.shutdown(); + for (IConnection conn : connections) { conn.close(); } - + try { clientScheduleService.awaitTermination(30, TimeUnit.SECONDS); } catch (InterruptedException e) { LOG.error("Error when shutting down client scheduler", e); } - + connections = null; //we need to release resources associated with client channel factory diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java index 169940f06a0..63c861ace74 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java @@ -133,8 +133,4 @@ private void writeTaskMessage(ChannelBufferOutputStream bout, TaskMessage messag if (payload_len >0) bout.write(message.message()); } - - public ArrayList getMsgs() { - return msgs; - } } \ No newline at end of file From c715e07a47b498b8a9207dbc853ca4015167b54b Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Thu, 4 Jun 2015 11:23:59 +0100 Subject: [PATCH 51/89] Use Netty's callback to perform the background flush --- .../storm/messaging/netty/Client.java | 164 +++++------------- .../storm/messaging/netty/MessageBatcher.java | 37 ++++ ...orHandler.java => StormClientHandler.java} | 20 ++- .../netty/StormClientPipelineFactory.java | 2 +- 4 files changed, 91 insertions(+), 132 deletions(-) create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java rename storm-core/src/jvm/backtype/storm/messaging/netty/{StormClientErrorHandler.java => StormClientHandler.java} (72%) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 0d754485a38..edb9f2b7344 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -65,6 +65,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private static final Logger LOG = LoggerFactory.getLogger(Client.class); private static final String PREFIX = "Netty-Client-"; private static final long NO_DELAY_MS = 0L; + private static final long MINIMUM_INITIAL_DELAY_MS = 30000L; private final StormBoundedExponentialBackoffRetry retryPolicy; private final ClientBootstrap bootstrap; @@ -113,18 +114,9 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { */ private volatile boolean closing = false; - /** - * How many messages should be batched together before sending them to the remote destination. - * - * Messages are batched to optimize network throughput at the expense of latency. - */ - private final int messageBatchSize; - private final HashedWheelTimer scheduler; - private final Object pendingMessageLock = new Object(); - private MessageBatch pendingMessage; - private Timeout pendingFlush; + private final MessageBatcher batcher; @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, HashedWheelTimer scheduler, String host, int port) { @@ -132,7 +124,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { this.scheduler = scheduler; int bufferSize = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE)); LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize); - messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); + int messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144); maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES)); int minWaitMs = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS)); @@ -145,9 +137,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { dstAddressPrefixedName = prefixedName(dstAddress); scheduleConnect(NO_DELAY_MS); - // Dummy values to avoid null checks - pendingMessage = new MessageBatch(messageBatchSize); - pendingFlush = scheduler.newTimeout(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); + batcher = new MessageBatcher(messageBatchSize); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -250,90 +240,28 @@ public void send(Iterator msgs) { return; } - MessageBatch replacement = new MessageBatch(messageBatchSize); - MessageBatch previous; - synchronized (pendingMessageLock) { - // pendingMessage is never null - previous = pendingMessage; - pendingMessage = replacement; - - // We are flushing the pending messages, therefore we can cancel the current pending flush - // The cancel is idempotent - pendingFlush.cancel(); - } - - // Collect messages into batches (to optimize network throughput) - Batches batches = createBatches(previous, msgs); - // Then flush the batches that are full - flushMessages(channel, batches.fullBatches); - - if (batches.unfilled.isEmpty()) { - // All messages ended up neatly into batches; there are no unfilled MessageBatch - return; + while (msgs.hasNext()) { + TaskMessage message = msgs.next(); + MessageBatch full = batcher.add(message); + if(full != null){ + flushMessages(channel, full); + } } - if (channel.isWritable()) { - // Netty's internal buffer is not full. We should write the unfilled MessageBatch immediately - // to reduce latency - flushMessages(channel, batches.unfilled); + if (!batcher.isEmpty() && channel.isWritable()) { + // Netty's internal buffer is not full and we still have message left in the batcher. + // We should write the unfilled MessageBatch immediately to reduce latency + MessageBatch batch = batcher.drain(); + flushMessages(channel, batch); } else { // We have an unfilled MessageBatch, but Netty's internal buffer is full, meaning that we have time. // In this situation, waiting for more messages before handing it to Netty yields better throughput - queueUp(channel, batches.unfilled); - } - } - - private void queueUp(Channel channel, MessageBatch unfilled) { - Batches batches; - synchronized (pendingMessageLock) { - batches = createBatches(pendingMessage, unfilled.getMsgs().iterator()); - pendingMessage = batches.unfilled; - - if(!pendingMessage.isEmpty()) { - // We have a MessageBatch that isn't full yet, so we will wait for more messages. - // However, we don't want to wait indefinitely so we schedule a timeout which flushes - // this batch if it's still not flushed after a delay - - // First, cancel the currently pending flush, because we just saw that Netty's - // buffer is full and thus we know we can wait longer - pendingFlush.cancel(); - - // Schedule the new flush - pendingFlush = scheduler.newTimeout(new Flush(pendingMessage), 10, TimeUnit.MILLISECONDS); - } - } - - // MessageBatches that were filled are immediately handed to Netty - flushMessages(channel, batches.fullBatches); - - } - - - private static class Batches { - final List fullBatches; - final MessageBatch unfilled; - - private Batches(List fullBatches, MessageBatch unfilled) { - this.fullBatches = fullBatches; - this.unfilled = unfilled; + // The messages are already in the buffer, and we know that the writability was false at that point + // Therefore we can rely on Netty's writability change. } } - private Batches createBatches(MessageBatch previous, Iterator msgs){ - List ret = new ArrayList(); - while (msgs.hasNext()) { - TaskMessage message = msgs.next(); - previous.add(message); - if (previous.isFull()) { - ret.add(previous); - previous = new MessageBatch(messageBatchSize); - } - } - - return new Batches(ret, previous); - } - private Channel getConnectedChannel() { Channel channel = channelRef.get(); if (connectionEstablished(channel)) { @@ -375,19 +303,16 @@ private int iteratorSize(Iterator msgs) { return size; } - private void flushMessages(Channel channel, List batches) { - for (MessageBatch batch : batches) { - flushMessages(channel, batch); - } - } - - /** * Asynchronously writes the message batch to the channel. * * If the write operation fails, then we will close the channel and trigger a reconnect. */ private void flushMessages(Channel channel, final MessageBatch batch) { + if(batch.isEmpty()){ + return; + } + final int numMessages = batch.size(); LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString()); pendingMessages.addAndGet(numMessages); @@ -489,39 +414,30 @@ public String toString() { * This task runs on a single thread shared among all clients, and thus * should not perform operations that block or are expensive. */ - private class Flush implements TimerTask { - private final MessageBatch instructor; - - private Flush(MessageBatch instructor) { - this.instructor = instructor; - } - + private final TimerTask FLUSH = new TimerTask() { @Override public void run(Timeout timeout) throws Exception { - MessageBatch toSend; - MessageBatch replacement = new MessageBatch(messageBatchSize); - synchronized (pendingMessageLock){ - if(instructor == pendingMessage){ - // It's still the batch which scheduled this timeout - toSend = pendingMessage; - pendingMessage = replacement; - checkState(!toSend.isFull(), "Only unfilled batches should get timeouts scheduled"); - } else { - // It's no longer the batch which scheduled this timeout - // No need to work on this one - toSend = null; - } - } + MessageBatch toSend = batcher.drain(); - if(toSend!=null){ - Channel channel = getConnectedChannel(); - if(channel == null) { - dropMessages(toSend); - } else { - flushMessages(channel, toSend); - } + Channel channel = getConnectedChannel(); + if(channel == null) { + dropMessages(toSend); + } else { + flushMessages(channel, toSend); } } + }; + + /** + * Called by Netty thread on change in channel interest + * @param channel + */ + public void channelInterestChanged(Channel channel) { + if(channel.isWritable()){ + // Channel is writable again + MessageBatch pending = batcher.drain(); + flushMessages(channel, pending); + } } /** diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java new file mode 100644 index 00000000000..e724a6deefa --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java @@ -0,0 +1,37 @@ +package backtype.storm.messaging.netty; + +import backtype.storm.messaging.TaskMessage; + +/** + * Enno Shioji + */ +public class MessageBatcher { + private final int mesageBatchSize; + private MessageBatch currentBatch; + + public MessageBatcher(int mesageBatchSize){ + this.mesageBatchSize = mesageBatchSize; + this.currentBatch = new MessageBatch(mesageBatchSize); + } + + public synchronized MessageBatch add(TaskMessage msg){ + currentBatch.add(msg); + if(currentBatch.isFull()){ + MessageBatch ret = currentBatch; + currentBatch = new MessageBatch(mesageBatchSize); + return ret; + } else { + return null; + } + } + + public synchronized boolean isEmpty() { + return currentBatch.isEmpty(); + } + + public synchronized MessageBatch drain() { + MessageBatch ret = currentBatch; + currentBatch = new MessageBatch(mesageBatchSize); + return ret; + } +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientErrorHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java similarity index 72% rename from storm-core/src/jvm/backtype/storm/messaging/netty/StormClientErrorHandler.java rename to storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java index ae317aa3f6c..b91a76dae44 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientErrorHandler.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java @@ -23,19 +23,25 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class StormClientErrorHandler extends SimpleChannelUpstreamHandler { - private static final Logger LOG = LoggerFactory.getLogger(StormClientErrorHandler.class); - private String name; +public class StormClientHandler extends SimpleChannelUpstreamHandler { + private static final Logger LOG = LoggerFactory.getLogger(StormClientHandler.class); + private Client client; - StormClientErrorHandler(String name) { - this.name = name; + StormClientHandler(Client client) { + this.client = client; + } + + @Override + public void channelInterestChanged(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { + + client.channelInterestChanged(e.getChannel()); } @Override public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent event) { Throwable cause = event.getCause(); if (!(cause instanceof ConnectException)) { - LOG.info("Connection failed " + name, cause); - } + LOG.info("Connection failed " + client.dstAddressPrefixedName, cause); + } } } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java index 73c50a18362..6bad8e386d5 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientPipelineFactory.java @@ -37,7 +37,7 @@ public ChannelPipeline getPipeline() throws Exception { // Encoder pipeline.addLast("encoder", new MessageEncoder()); // business logic. - pipeline.addLast("handler", new StormClientErrorHandler(client.dstAddressPrefixedName)); + pipeline.addLast("handler", new StormClientHandler(client)); return pipeline; } From 5e134973eadda2c76b1fd62a0f93281abf326688 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Thu, 4 Jun 2015 15:58:04 +0100 Subject: [PATCH 52/89] Clean up --- .../storm/messaging/netty/Client.java | 50 +++++----------- .../storm/messaging/netty/Context.java | 4 +- .../storm/messaging/netty/MessageBatch.java | 26 +-------- .../storm/messaging/netty/MessageBatcher.java | 37 ------------ .../storm/messaging/netty/MessageBuffer.java | 58 +++++++++++++++++++ .../messaging/netty/StormClientHandler.java | 3 +- 6 files changed, 77 insertions(+), 101 deletions(-) delete mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java create mode 100644 storm-core/src/jvm/backtype/storm/messaging/netty/MessageBuffer.java diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index edb9f2b7344..c779733d25b 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -65,7 +65,6 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private static final Logger LOG = LoggerFactory.getLogger(Client.class); private static final String PREFIX = "Netty-Client-"; private static final long NO_DELAY_MS = 0L; - private static final long MINIMUM_INITIAL_DELAY_MS = 30000L; private final StormBoundedExponentialBackoffRetry retryPolicy; private final ClientBootstrap bootstrap; @@ -116,7 +115,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private final HashedWheelTimer scheduler; - private final MessageBatcher batcher; + private final MessageBuffer batcher; @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, HashedWheelTimer scheduler, String host, int port) { @@ -136,8 +135,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { dstAddress = new InetSocketAddress(host, port); dstAddressPrefixedName = prefixedName(dstAddress); scheduleConnect(NO_DELAY_MS); - - batcher = new MessageBatcher(messageBatchSize); + batcher = new MessageBuffer(messageBatchSize); } private ClientBootstrap createClientBootstrap(ChannelFactory factory, int bufferSize) { @@ -249,16 +247,18 @@ public void send(Iterator msgs) { } } - if (!batcher.isEmpty() && channel.isWritable()) { - // Netty's internal buffer is not full and we still have message left in the batcher. + if(channel.isWritable()){ + // Netty's internal buffer is not full and we still have message left in the buffer. // We should write the unfilled MessageBatch immediately to reduce latency MessageBatch batch = batcher.drain(); - flushMessages(channel, batch); + if(batch != null) { + flushMessages(channel, batch); + } } else { - // We have an unfilled MessageBatch, but Netty's internal buffer is full, meaning that we have time. - // In this situation, waiting for more messages before handing it to Netty yields better throughput - // The messages are already in the buffer, and we know that the writability was false at that point - // Therefore we can rely on Netty's writability change. + // Channel's buffer is full, meaning that we have time to wait other messages to arrive, and create a bigger + // batch. This yields better throughput. + // We can rely on `notifyInterestChanged` to push these messages as soon as there is spece in Netty's buffer + // because we know `Channel.isWritable` was false after the messages were already in the buffer. } } @@ -288,10 +288,6 @@ private void dropMessages(Iterator msgs) { messagesLost.getAndAdd(msgCount); } - private void dropMessages(MessageBatch msgs) { - messagesLost.getAndAdd(msgs.size()); - } - private int iteratorSize(Iterator msgs) { int size = 0; if (msgs != null) { @@ -408,33 +404,13 @@ public String toString() { return String.format("Netty client for connecting to %s", dstAddressPrefixedName); } - /** - * Asynchronously flushes pending messages to the remote address, if they have not been - * flushed by other means. - * This task runs on a single thread shared among all clients, and thus - * should not perform operations that block or are expensive. - */ - private final TimerTask FLUSH = new TimerTask() { - @Override - public void run(Timeout timeout) throws Exception { - MessageBatch toSend = batcher.drain(); - - Channel channel = getConnectedChannel(); - if(channel == null) { - dropMessages(toSend); - } else { - flushMessages(channel, toSend); - } - } - }; - /** * Called by Netty thread on change in channel interest * @param channel */ - public void channelInterestChanged(Channel channel) { + public void notifyInterestChanged(Channel channel) { if(channel.isWritable()){ - // Channel is writable again + // Channel is writable again, write if there are any messages pending MessageBatch pending = batcher.drain(); flushMessages(channel, pending); } diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java index 64f67ba57fc..bce02acc3f3 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Context.java @@ -62,8 +62,8 @@ public void prepare(Map storm_conf) { clientChannelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(bossFactory), Executors.newCachedThreadPool(workerFactory)); } - - clientScheduleService = new HashedWheelTimer(new NettyRenameThreadFactory("client-schedule-timer"), 10, TimeUnit.MILLISECONDS); + + clientScheduleService = new HashedWheelTimer(new NettyRenameThreadFactory("client-schedule-timer")); } /** diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java index 169940f06a0..ec0dc0f4077 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatch.java @@ -35,32 +35,15 @@ class MessageBatch { encoded_length = ControlMessage.EOB_MESSAGE.encodeLength(); } - void add(TaskMessage obj) { - if (obj == null) - throw new RuntimeException("null object forbidded in message batch"); + void add(TaskMessage msg) { + if (msg == null) + throw new RuntimeException("null object forbidden in message batch"); - TaskMessage msg = (TaskMessage)obj; msgs.add(msg); encoded_length += msgEncodeLength(msg); } - TaskMessage get(int index) { - return msgs.get(index); - } - - /** - * try to add a TaskMessage to a batch - * @param taskMsg - * @return false if the msg could not be added due to buffer size limit; true otherwise - */ - boolean tryAdd(TaskMessage taskMsg) { - if ((encoded_length + msgEncodeLength(taskMsg)) > buffer_size) - return false; - add(taskMsg); - return true; - } - private int msgEncodeLength(TaskMessage taskMsg) { if (taskMsg == null) return 0; @@ -134,7 +117,4 @@ private void writeTaskMessage(ChannelBufferOutputStream bout, TaskMessage messag bout.write(message.message()); } - public ArrayList getMsgs() { - return msgs; - } } \ No newline at end of file diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java deleted file mode 100644 index e724a6deefa..00000000000 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBatcher.java +++ /dev/null @@ -1,37 +0,0 @@ -package backtype.storm.messaging.netty; - -import backtype.storm.messaging.TaskMessage; - -/** - * Enno Shioji - */ -public class MessageBatcher { - private final int mesageBatchSize; - private MessageBatch currentBatch; - - public MessageBatcher(int mesageBatchSize){ - this.mesageBatchSize = mesageBatchSize; - this.currentBatch = new MessageBatch(mesageBatchSize); - } - - public synchronized MessageBatch add(TaskMessage msg){ - currentBatch.add(msg); - if(currentBatch.isFull()){ - MessageBatch ret = currentBatch; - currentBatch = new MessageBatch(mesageBatchSize); - return ret; - } else { - return null; - } - } - - public synchronized boolean isEmpty() { - return currentBatch.isEmpty(); - } - - public synchronized MessageBatch drain() { - MessageBatch ret = currentBatch; - currentBatch = new MessageBatch(mesageBatchSize); - return ret; - } -} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBuffer.java b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBuffer.java new file mode 100644 index 00000000000..d485e3a189d --- /dev/null +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/MessageBuffer.java @@ -0,0 +1,58 @@ +/** + * 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 backtype.storm.messaging.netty; + +import backtype.storm.messaging.TaskMessage; + +/** + * Encapsulates the state used for batching up messages. + */ +public class MessageBuffer { + private final int mesageBatchSize; + private MessageBatch currentBatch; + + public MessageBuffer(int mesageBatchSize){ + this.mesageBatchSize = mesageBatchSize; + this.currentBatch = new MessageBatch(mesageBatchSize); + } + + public synchronized MessageBatch add(TaskMessage msg){ + currentBatch.add(msg); + if(currentBatch.isFull()){ + MessageBatch ret = currentBatch; + currentBatch = new MessageBatch(mesageBatchSize); + return ret; + } else { + return null; + } + } + + public synchronized boolean isEmpty() { + return currentBatch.isEmpty(); + } + + public synchronized MessageBatch drain() { + if(!currentBatch.isEmpty()) { + MessageBatch ret = currentBatch; + currentBatch = new MessageBatch(mesageBatchSize); + return ret; + } else { + return null; + } + } +} diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java index b91a76dae44..2d25001bb0b 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/StormClientHandler.java @@ -33,8 +33,7 @@ public class StormClientHandler extends SimpleChannelUpstreamHandler { @Override public void channelInterestChanged(ChannelHandlerContext ctx, ChannelStateEvent e) throws Exception { - - client.channelInterestChanged(e.getChannel()); + client.notifyInterestChanged(e.getChannel()); } @Override From 6d6c260da3663f1ab1dfee6c9333fcdc278e9b46 Mon Sep 17 00:00:00 2001 From: Enno Shioji Date: Thu, 4 Jun 2015 16:01:56 +0100 Subject: [PATCH 53/89] Bring back graceful shutdown --- .../storm/messaging/netty/Client.java | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index c779733d25b..166c1b2925c 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -61,6 +61,8 @@ * the remote destination is currently unavailable. */ public class Client extends ConnectionWithStatus implements IStatefulObject { + private static final long PENDING_MESSAGES_FLUSH_TIMEOUT_MS = 600000L; + private static final long PENDING_MESSAGES_FLUSH_INTERVAL_MS = 1000L; private static final Logger LOG = LoggerFactory.getLogger(Client.class); private static final String PREFIX = "Netty-Client-"; @@ -357,11 +359,33 @@ public void close() { LOG.info("closing Netty Client {}", dstAddressPrefixedName); // Set closing to true to prevent any further reconnection attempts. closing = true; - + waitForPendingMessagesToBeSent(); closeChannel(); } } + private void waitForPendingMessagesToBeSent() { + LOG.info("waiting up to {} ms to send {} pending messages to {}", + PENDING_MESSAGES_FLUSH_TIMEOUT_MS, pendingMessages.get(), dstAddressPrefixedName); + long totalPendingMsgs = pendingMessages.get(); + long startMs = System.currentTimeMillis(); + while (pendingMessages.get() != 0) { + try { + long deltaMs = System.currentTimeMillis() - startMs; + if (deltaMs > PENDING_MESSAGES_FLUSH_TIMEOUT_MS) { + LOG.error("failed to send all pending messages to {} within timeout, {} of {} messages were not " + + "sent", dstAddressPrefixedName, pendingMessages.get(), totalPendingMsgs); + break; + } + Thread.sleep(PENDING_MESSAGES_FLUSH_INTERVAL_MS); + } + catch (InterruptedException e) { + break; + } + } + + } + private void closeChannel() { Channel channel = channelRef.get(); From 367a08d1b4ab434e42f824530b157c8e55160d19 Mon Sep 17 00:00:00 2001 From: Derek Dagit Date: Thu, 26 Mar 2015 16:30:03 -0500 Subject: [PATCH 54/89] Put emitted and transferred stats under correct columns --- storm-core/src/ui/public/templates/component-page-template.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/ui/public/templates/component-page-template.html b/storm-core/src/ui/public/templates/component-page-template.html index 1e916e37458..bfa572b16f0 100644 --- a/storm-core/src/ui/public/templates/component-page-template.html +++ b/storm-core/src/ui/public/templates/component-page-template.html @@ -91,8 +91,8 @@

Spout stats

{{#spoutSummary}} {{windowPretty}} - {{transferred}} {{emitted}} + {{transferred}} {{completeLatency}} {{acked}} {{failed}} From ff42a3debeeb0e0e1f0fc5b2a31179c4c06fabb7 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 14 Jul 2015 13:50:40 -0500 Subject: [PATCH 55/89] Added STORM-763 and STORM-839 to Changelog --- CHANGELOG.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8d242b7703f..d3dfbe14074 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,8 +1,11 @@ +## 0.9.6 + * STORM-763: nimbus reassigned worker A to another machine, but other worker's netty client can't connect to the new worker A + ## 0.9.5 * STORM-790: Log "task is null" instead of let worker died when task is null in transfer-fn * STORM-796: Add support for "error" command in ShellSpout * STORM-745: fix storm.cmd to evaluate 'shift' correctly with 'storm jar' - * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. + * STORM-130: Supervisor getting killed due to java.io.FileNotFoundException: File '../stormconf.ser' does not exist. ## 0.9.4 * STORM-559: ZkHosts in README should use 2181 as port. From 9221dba1eee59686ec6a1bcfd93d4ce542b071e9 Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Thu, 16 Jul 2015 10:06:28 +0800 Subject: [PATCH 56/89] Fix incorrect WARN log --- external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java b/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java index 918da74539f..df33c4b7413 100644 --- a/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java +++ b/external/storm-kafka/src/jvm/storm/kafka/KafkaUtils.java @@ -180,9 +180,7 @@ public static ByteBufferMessageSet fetchMessages(KafkaConfig config, SimpleConsu if (fetchResponse.hasError()) { KafkaError error = KafkaError.getError(fetchResponse.errorCode(topic, partitionId)); if (error.equals(KafkaError.OFFSET_OUT_OF_RANGE) && config.useStartOffsetTimeIfOffsetOutOfRange) { - LOG.warn("Got fetch request with offset out of range: [" + offset + "]; " + - "retrying with default start offset time from configuration. " + - "configured start offset time: [" + config.startOffsetTime + "]"); + LOG.warn("Got fetch request with offset out of range: [" + offset + "]"); throw new UpdateOffsetException(); } else { String message = "Error fetching data from [" + partition + "] for topic [" + topic + "]: [" + error + "]"; From a6557a565429b2e12fc0659c6484fc50f90b5cd5 Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Thu, 16 Jul 2015 10:19:41 +0800 Subject: [PATCH 57/89] Remove outdated failed offsets --- .../src/jvm/storm/kafka/PartitionManager.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java index d24a49e1831..001c5dfcfe9 100644 --- a/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java +++ b/external/storm-kafka/src/jvm/storm/kafka/PartitionManager.java @@ -164,6 +164,31 @@ private void fill() { _emittedToOffset = KafkaUtils.getOffset(_consumer, _spoutConfig.topic, _partition.partition, _spoutConfig); LOG.warn("Using new offset: {}", _emittedToOffset); // fetch failed, so don't update the metrics + + //fix bug [STORM-643] : remove outdated failed offsets + if (had_failed) { + // For the case of EarliestTime it would be better to discard + // all the failed offsets, that are earlier than actual EarliestTime + // offset, since they are anyway not there. + // These calls to broker API will be then saved. + + // In case of LatestTime - it is a question, if we still need to try out and + // reach those that are failed (they still may be available). + // But, by moving to LatestTime we are discarding messages in kafka queue. + // Since it is configured so, assume that it is ok for user to loose information + // and user cares about newest messages first. + // It makes sense not to do exceptions for those that are failed and discard them as well. + + SortedSet omitted = failed.headSet(_emittedToOffset); + + // Use tail, since sortedSet maintains its elements in ascending order + // Using tailSet will set a 'range' on original implementation + // so we couldn't then add objects that are out of range. + // For that reason we copy tail into new Set, where range is not set. + failed = new TreeSet(failed.tailSet(_emittedToOffset)); + LOG.warn("Removing the failed offsets that are out of range: {}", omitted); + } + return; } long end = System.nanoTime(); From 979e23cd73bbbb55ccadf9df9d62ac60b27e063e Mon Sep 17 00:00:00 2001 From: errordaiwa Date: Thu, 9 Jul 2015 10:40:20 +0800 Subject: [PATCH 58/89] fix issue of high cpu usage when bolt idle --- storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java index 932af164ae4..1f2110de35d 100644 --- a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java +++ b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java @@ -94,7 +94,7 @@ public void haltWithInterrupt() { public void consumeBatchWhenAvailable(EventHandler handler) { try { final long nextSequence = _consumer.get() + 1; - final long availableSequence = _barrier.waitFor(nextSequence, 10, TimeUnit.MILLISECONDS); + final long availableSequence = _barrier.waitFor(nextSequence, 1000, TimeUnit.MILLISECONDS); if(availableSequence >= nextSequence) { consumeBatchToCursor(availableSequence, handler); } From 449fb8cbfdbd5bce14bc3a47f9e0a91b617c49f8 Mon Sep 17 00:00:00 2001 From: errordaiwa Date: Fri, 10 Jul 2015 11:00:28 +0800 Subject: [PATCH 59/89] make disruptor queue wait timeout configurable Conflicts: conf/defaults.yaml --- conf/defaults.yaml | 1 + storm-core/src/clj/backtype/storm/daemon/executor.clj | 1 + storm-core/src/clj/backtype/storm/daemon/worker.clj | 2 ++ storm-core/src/clj/backtype/storm/disruptor.clj | 4 ++-- storm-core/src/jvm/backtype/storm/Config.java | 7 +++++++ .../src/jvm/backtype/storm/utils/DisruptorQueue.java | 8 ++++++-- .../test/jvm/backtype/storm/utils/DisruptorQueueTest.java | 2 +- 7 files changed, 20 insertions(+), 5 deletions(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index e2b33000aab..c05fd9eb9f8 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -153,5 +153,6 @@ topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSeria topology.trident.batch.emit.interval.millis: 500 topology.classpath: null topology.environment: null +topology.disruptor.wait.timeout.millis: 10 dev.zookeeper.path: "/tmp/dev-storm-zookeeper" diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj index 8acd459d15f..df8219b3f42 100644 --- a/storm-core/src/clj/backtype/storm/daemon/executor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj @@ -208,6 +208,7 @@ batch-transfer->worker (disruptor/disruptor-queue (str "executor" executor-id "-send-queue") (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE) + (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS) :claim-strategy :single-threaded :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY)) ] diff --git a/storm-core/src/clj/backtype/storm/daemon/worker.clj b/storm-core/src/clj/backtype/storm/daemon/worker.clj index d5569a77b29..08b9f000567 100644 --- a/storm-core/src/clj/backtype/storm/daemon/worker.clj +++ b/storm-core/src/clj/backtype/storm/daemon/worker.clj @@ -140,6 +140,7 @@ ;; TODO: this depends on the type of executor (map (fn [e] [e (disruptor/disruptor-queue (str "receive-queue" e) (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE) + (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS) :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY))])) (into {}) )) @@ -184,6 +185,7 @@ storm-conf (read-supervisor-storm-conf conf storm-id) executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions)) transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE) + (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS) :wait-strategy (storm-conf TOPOLOGY-DISRUPTOR-WAIT-STRATEGY)) executor-receive-queue-map (mk-receive-queue-map storm-conf executors) diff --git a/storm-core/src/clj/backtype/storm/disruptor.clj b/storm-core/src/clj/backtype/storm/disruptor.clj index a723601c7a4..e96e49df42a 100644 --- a/storm-core/src/clj/backtype/storm/disruptor.clj +++ b/storm-core/src/clj/backtype/storm/disruptor.clj @@ -45,10 +45,10 @@ ;; wouldn't make it to the acker until the batch timed out and another tuple was played into the queue, ;; unblocking the consumer (defnk disruptor-queue - [^String queue-name buffer-size :claim-strategy :multi-threaded :wait-strategy :block] + [^String queue-name buffer-size timeout :claim-strategy :multi-threaded :wait-strategy :block] (DisruptorQueue. queue-name ((CLAIM-STRATEGY claim-strategy) buffer-size) - (mk-wait-strategy wait-strategy))) + (mk-wait-strategy wait-strategy) timeout)) (defn clojure-handler [afn] diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 77e7cbc233a..8ce9c791954 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -859,6 +859,13 @@ public class Config extends HashMap { public static final String ISOLATION_SCHEDULER_MACHINES = "isolation.scheduler.machines"; public static final Object ISOLATION_SCHEDULER_MACHINES_SCHEMA = Map.class; + /** + * Configure timeout milliseconds used for disruptor queue wait strategy. Can be used to tradeoff latency + * vs. CPU usage + */ + public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis"; + public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = Number.class; + public static void setClasspath(Map conf, String cp) { conf.put(Config.TOPOLOGY_CLASSPATH, cp); } diff --git a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java index 1f2110de35d..195c2f74783 100644 --- a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java +++ b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java @@ -60,8 +60,10 @@ public class DisruptorQueue implements IStatefulObject { private static String PREFIX = "disruptor-"; private String _queueName = ""; + + private long _waitTimeout; - public DisruptorQueue(String queueName, ClaimStrategy claim, WaitStrategy wait) { + public DisruptorQueue(String queueName, ClaimStrategy claim, WaitStrategy wait, long timeout) { this._queueName = PREFIX + queueName; _buffer = new RingBuffer(new ObjectEventFactory(), claim, wait); _consumer = new Sequence(); @@ -77,6 +79,8 @@ public DisruptorQueue(String queueName, ClaimStrategy claim, WaitStrategy wait) throw new RuntimeException("This code should be unreachable!", e); } } + + _waitTimeout = timeout; } public String getName() { @@ -94,7 +98,7 @@ public void haltWithInterrupt() { public void consumeBatchWhenAvailable(EventHandler handler) { try { final long nextSequence = _consumer.get() + 1; - final long availableSequence = _barrier.waitFor(nextSequence, 1000, TimeUnit.MILLISECONDS); + final long availableSequence = _barrier.waitFor(nextSequence, _waitTimeout, TimeUnit.MILLISECONDS); if(availableSequence >= nextSequence) { consumeBatchToCursor(availableSequence, handler); } diff --git a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java index 653fd33daea..9ddb37a6dfe 100644 --- a/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java +++ b/storm-core/test/jvm/backtype/storm/utils/DisruptorQueueTest.java @@ -148,6 +148,6 @@ public void run() { private static DisruptorQueue createQueue(String name, int queueSize) { return new DisruptorQueue(name, new MultiThreadedClaimStrategy( - queueSize), new BlockingWaitStrategy()); + queueSize), new BlockingWaitStrategy(), 10L); } } From 511ba02a9fd7a5bffbc027501a307d7bca8871cc Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 17 Jul 2015 08:20:30 +0900 Subject: [PATCH 60/89] use IntegerValidator to verify disruptor queue wait timeout config --- storm-core/src/jvm/backtype/storm/Config.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 8ce9c791954..993052d963c 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -864,7 +864,7 @@ public class Config extends HashMap { * vs. CPU usage */ public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis"; - public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = Number.class; + public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = ConfigValidation.IntegerValidator; public static void setClasspath(Map conf, String cp) { conf.put(Config.TOPOLOGY_CLASSPATH, cp); From b3c5dc99a46d2f045cc7843732d5711aacefef32 Mon Sep 17 00:00:00 2001 From: errordaiwa Date: Mon, 13 Jul 2015 17:46:16 +0800 Subject: [PATCH 61/89] update version of Disruptor queue to 2.10.4, consumer of queue will use waitfor method without timeout Conflicts: storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java --- pom.xml | 2 +- storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 80fa42b2794..105838d8f5a 100644 --- a/pom.xml +++ b/pom.xml @@ -196,7 +196,7 @@ 1.11 4.3.3 0.2.4 - 2.10.1 + 2.10.4 0.9.0 16.0.1 1.0.13 diff --git a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java index 195c2f74783..c4c936a134c 100644 --- a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java +++ b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java @@ -98,7 +98,7 @@ public void haltWithInterrupt() { public void consumeBatchWhenAvailable(EventHandler handler) { try { final long nextSequence = _consumer.get() + 1; - final long availableSequence = _barrier.waitFor(nextSequence, _waitTimeout, TimeUnit.MILLISECONDS); + final long availableSequence = _barrier.waitFor(nextSequence); if(availableSequence >= nextSequence) { consumeBatchToCursor(availableSequence, handler); } From fb2e9690c718b7448ef59e05a9e8bfb6afa703de Mon Sep 17 00:00:00 2001 From: errordaiwa Date: Tue, 14 Jul 2015 12:40:45 +0800 Subject: [PATCH 62/89] Merge branch 'STORM-929' into STORM-935 Conflicts: conf/defaults.yaml --- storm-core/src/jvm/backtype/storm/Config.java | 7 +++++++ .../src/jvm/backtype/storm/utils/DisruptorQueue.java | 7 +++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index 993052d963c..ff8b59591b7 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -866,6 +866,13 @@ public class Config extends HashMap { public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis"; public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = ConfigValidation.IntegerValidator; + /** + * Configure timeout milliseconds used for disruptor queue wait strategy. Can be used to tradeoff latency + * vs. CPU usage + */ + public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis"; + public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator; + public static void setClasspath(Map conf, String cp) { conf.put(Config.TOPOLOGY_CLASSPATH, cp); } diff --git a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java index c4c936a134c..4d3f18bbb4e 100644 --- a/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java +++ b/storm-core/src/jvm/backtype/storm/utils/DisruptorQueue.java @@ -98,8 +98,11 @@ public void haltWithInterrupt() { public void consumeBatchWhenAvailable(EventHandler handler) { try { final long nextSequence = _consumer.get() + 1; - final long availableSequence = _barrier.waitFor(nextSequence); - if(availableSequence >= nextSequence) { + final long availableSequence = + _waitTimeout == 0L ? _barrier.waitFor(nextSequence) : _barrier.waitFor(nextSequence, _waitTimeout, + TimeUnit.MILLISECONDS); + + if (availableSequence >= nextSequence) { consumeBatchToCursor(availableSequence, handler); } } catch (AlertException e) { From cfe25b0cc6f0839ed349d60b0016bd353ac51bc0 Mon Sep 17 00:00:00 2001 From: errordaiwa Date: Thu, 16 Jul 2015 09:02:17 +0800 Subject: [PATCH 63/89] change default disruptor queue timeout to 1000ms Conflicts: conf/defaults.yaml --- conf/defaults.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/defaults.yaml b/conf/defaults.yaml index c05fd9eb9f8..06819550937 100644 --- a/conf/defaults.yaml +++ b/conf/defaults.yaml @@ -153,6 +153,6 @@ topology.tuple.serializer: "backtype.storm.serialization.types.ListDelegateSeria topology.trident.batch.emit.interval.millis: 500 topology.classpath: null topology.environment: null -topology.disruptor.wait.timeout.millis: 10 +topology.disruptor.wait.timeout.millis: 1000 dev.zookeeper.path: "/tmp/dev-storm-zookeeper" From daba5976a1dbdbb1c222bbbd0385706a265edf4c Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 17 Jul 2015 08:30:35 +0900 Subject: [PATCH 64/89] Fix cherry-pick/merge issue --- storm-core/src/jvm/backtype/storm/Config.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/Config.java b/storm-core/src/jvm/backtype/storm/Config.java index ff8b59591b7..993052d963c 100644 --- a/storm-core/src/jvm/backtype/storm/Config.java +++ b/storm-core/src/jvm/backtype/storm/Config.java @@ -866,13 +866,6 @@ public class Config extends HashMap { public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis"; public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = ConfigValidation.IntegerValidator; - /** - * Configure timeout milliseconds used for disruptor queue wait strategy. Can be used to tradeoff latency - * vs. CPU usage - */ - public static final String TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS="topology.disruptor.wait.timeout.millis"; - public static final Object TOPOLOGY_DISRUPTOR_WAIT_TIMEOUT_MILLIS_SCHEMA = ConfigValidation.NotNullPosIntegerValidator; - public static void setClasspath(Map conf, String cp) { conf.put(Config.TOPOLOGY_CLASSPATH, cp); } From f17b4151fc6f5dab1c6bd091a3e3b314ba0f0ad7 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 17 Jul 2015 08:31:46 +0900 Subject: [PATCH 65/89] add STORM-935 and STORM-503 to CHANGELOG.md --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index d3dfbe14074..761d5150324 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,7 @@ ## 0.9.6 * STORM-763: nimbus reassigned worker A to another machine, but other worker's netty client can't connect to the new worker A + * STORM-935: Update Disruptor queue version to 2.10.4 + * STORM-503: Short disruptor queue wait time leads to high CPU usage when idle ## 0.9.5 * STORM-790: Log "task is null" instead of let worker died when task is null in transfer-fn From 12b4459b0487106495ec49326c640ffebc6cbbd4 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 17 Jul 2015 08:56:30 +0900 Subject: [PATCH 66/89] add STORM-728 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 761d5150324..d562db5bf23 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,7 @@ * STORM-763: nimbus reassigned worker A to another machine, but other worker's netty client can't connect to the new worker A * STORM-935: Update Disruptor queue version to 2.10.4 * STORM-503: Short disruptor queue wait time leads to high CPU usage when idle + * STORM-728: Put emitted and transferred stats under correct columns ## 0.9.5 * STORM-790: Log "task is null" instead of let worker died when task is null in transfer-fn From 67cc2201a3a2a4f5cb724d0c350a39c2c90f7e5a Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Sat, 18 Jul 2015 10:10:28 +0900 Subject: [PATCH 67/89] add STORM-643 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index d562db5bf23..faa310bac46 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,7 @@ * STORM-935: Update Disruptor queue version to 2.10.4 * STORM-503: Short disruptor queue wait time leads to high CPU usage when idle * STORM-728: Put emitted and transferred stats under correct columns + * STORM-643: KafkaUtils repeatedly fetches messages whose offset is out of range ## 0.9.5 * STORM-790: Log "task is null" instead of let worker died when task is null in transfer-fn From d222c312ad393272fb6a59c3bd56b60536e66fec Mon Sep 17 00:00:00 2001 From: caofangkun Date: Fri, 31 Jul 2015 10:36:42 +0800 Subject: [PATCH 68/89] STORM-933:NullPointerException during KafkaSpout deactivation --- external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java b/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java index 0a8fe36377d..3af648c59aa 100644 --- a/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java +++ b/external/storm-kafka/src/jvm/storm/kafka/ZkCoordinator.java @@ -33,7 +33,7 @@ public class ZkCoordinator implements PartitionCoordinator { int _totalTasks; String _topologyInstanceId; Map _managers = new HashMap(); - List _cachedList; + List _cachedList = new ArrayList(); Long _lastRefreshTime = null; int _refreshFreqMs; DynamicPartitionConnections _connections; From a58e30cbbccd2c8159544040a92cafd841a49df2 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 3 Aug 2015 07:59:05 +0900 Subject: [PATCH 69/89] add STORM-933 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index faa310bac46..57924a57516 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,7 @@ * STORM-503: Short disruptor queue wait time leads to high CPU usage when idle * STORM-728: Put emitted and transferred stats under correct columns * STORM-643: KafkaUtils repeatedly fetches messages whose offset is out of range + * STORM-933: NullPointerException during KafkaSpout deactivation ## 0.9.5 * STORM-790: Log "task is null" instead of let worker died when task is null in transfer-fn From e696481c164f011e59396ba2c7bb1031db31381d Mon Sep 17 00:00:00 2001 From: Abhishek Agarwal Date: Tue, 8 Sep 2015 16:50:39 +0530 Subject: [PATCH 70/89] Backporting STORM-1027 --- .../clj/backtype/storm/daemon/executor.clj | 72 ++++++++++--------- 1 file changed, 39 insertions(+), 33 deletions(-) diff --git a/storm-core/src/clj/backtype/storm/daemon/executor.clj b/storm-core/src/clj/backtype/storm/daemon/executor.clj index df8219b3f42..1d2fe651c74 100644 --- a/storm-core/src/clj/backtype/storm/daemon/executor.clj +++ b/storm-core/src/clj/backtype/storm/daemon/executor.clj @@ -274,27 +274,32 @@ receive-queue [[nil (TupleImpl. worker-context [interval] Constants/SYSTEM_TASK_ID Constants/METRICS_TICK_STREAM_ID)]])))))) -(defn metrics-tick [executor-data task-data ^TupleImpl tuple] - (let [{:keys [interval->task->metric-registry ^WorkerTopologyContext worker-context]} executor-data - interval (.getInteger tuple 0) - 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)) - (.getThisWorkerPort worker-context) - (:component-id executor-data) - task-id - (long (/ (System/currentTimeMillis) 1000)) - interval) - data-points (->> name->imetric - (map (fn [[name imetric]] - (let [value (.getValueAndReset ^IMetric imetric)] - (if value - (IMetricsConsumer$DataPoint. name value))))) - (filter identity) - (into []))] - (if (seq data-points) - (task/send-unanchored task-data Constants/METRICS_STREAM_ID [task-info data-points])))) +(defn metrics-tick + ([executor-data task-data ^TupleImpl tuple overflow-buffer] + (let [{:keys [interval->task->metric-registry ^WorkerTopologyContext worker-context]} executor-data + interval (.getInteger tuple 0) + 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)) + (.getThisWorkerPort worker-context) + (:component-id executor-data) + task-id + (long (/ (System/currentTimeMillis) 1000)) + interval) + data-points (->> name->imetric + (map (fn [[name imetric]] + (let [value (.getValueAndReset ^IMetric imetric)] + (if value + (IMetricsConsumer$DataPoint. name value))))) + (filter identity) + (into []))] + (if (seq data-points) + (task/send-unanchored task-data Constants/METRICS_STREAM_ID [task-info data-points] overflow-buffer)))) + ([executor-data task-data ^TupleImpl tuple] + (metrics-tick executor-data task-data tuple nil) + )) + (defn setup-ticks! [worker executor-data] (let [storm-conf (:storm-conf executor-data) @@ -424,7 +429,16 @@ last-active (atom false) spouts (ArrayList. (map :object (vals task-datas))) rand (Random. (Utils/secureRandomLong)) - + + ;; the overflow buffer is used to ensure that spouts never block when emitting + ;; this ensures that the spout can always clear the incoming buffer (acks and fails), which + ;; prevents deadlock from occuring across the topology (e.g. Spout -> Bolt -> Acker -> Spout, and all + ;; buffers filled up) + ;; when the overflow buffer is full, spouts stop calling nextTuple until it's able to clear the overflow buffer + ;; this limits the size of the overflow buffer to however many tuples a spout emits in one call of nextTuple, + ;; preventing memory issues + overflow-buffer (LinkedList.) + pending (RotatingMap. 2 ;; microoptimize for performance of .size method (reify RotatingMap$ExpiredCallback @@ -436,7 +450,7 @@ (let [stream-id (.getSourceStreamId tuple)] (condp = stream-id Constants/SYSTEM_TICK_STREAM_ID (.rotate pending) - Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple) + Constants/METRICS_TICK_STREAM_ID (metrics-tick executor-data (get task-datas task-id) tuple overflow-buffer) (let [id (.getValue tuple 0) [stored-task-id spout-id tuple-finished-info start-time-ms] (.remove pending id)] (when spout-id @@ -455,16 +469,8 @@ event-handler (mk-task-receiver executor-data tuple-action-fn) has-ackers? (has-ackers? storm-conf) emitted-count (MutableLong. 0) - empty-emit-streak (MutableLong. 0) - - ;; the overflow buffer is used to ensure that spouts never block when emitting - ;; this ensures that the spout can always clear the incoming buffer (acks and fails), which - ;; prevents deadlock from occuring across the topology (e.g. Spout -> Bolt -> Acker -> Spout, and all - ;; buffers filled up) - ;; when the overflow buffer is full, spouts stop calling nextTuple until it's able to clear the overflow buffer - ;; this limits the size of the overflow buffer to however many tuples a spout emits in one call of nextTuple, - ;; preventing memory issues - overflow-buffer (LinkedList.)] + empty-emit-streak (MutableLong. 0)] + [(async-loop (fn [] From aa8114f15963b80dabaf97466334296ca2b4c51d Mon Sep 17 00:00:00 2001 From: Michael Schonfeld Date: Fri, 18 Sep 2015 12:31:32 -0400 Subject: [PATCH 71/89] [STORM-1051] Fix for flushMessagse NPE --- storm-core/src/jvm/backtype/storm/messaging/netty/Client.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 166c1b2925c..7f801e5075b 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -307,7 +307,7 @@ private int iteratorSize(Iterator msgs) { * If the write operation fails, then we will close the channel and trigger a reconnect. */ private void flushMessages(Channel channel, final MessageBatch batch) { - if(batch.isEmpty()){ + if(null == batch || batch.isEmpty()){ return; } From e6819387019dca33e8d835a1110ca78fd8d8b672 Mon Sep 17 00:00:00 2001 From: Michael Schonfeld Date: Sat, 19 Sep 2015 01:55:09 -0400 Subject: [PATCH 72/89] match style to the rest of the file --- storm-core/src/jvm/backtype/storm/messaging/netty/Client.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 7f801e5075b..067dddc4f12 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -307,7 +307,7 @@ private int iteratorSize(Iterator msgs) { * If the write operation fails, then we will close the channel and trigger a reconnect. */ private void flushMessages(Channel channel, final MessageBatch batch) { - if(null == batch || batch.isEmpty()){ + if (null == batch || batch.isEmpty()) { return; } From 69c0a28624b0ce13afbc38a28a7d06ba5aa01af9 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Mon, 21 Sep 2015 07:02:26 +0900 Subject: [PATCH 73/89] add STORM-1051 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 57924a57516..03a677fdae8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.6 + * STORM-1051: Netty Client.java's flushMessages produces a NullPointerException * STORM-763: nimbus reassigned worker A to another machine, but other worker's netty client can't connect to the new worker A * STORM-935: Update Disruptor queue version to 2.10.4 * STORM-503: Short disruptor queue wait time leads to high CPU usage when idle From 1730fd3bd9c475e7add4cddbdca4445867b37060 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 24 Sep 2015 09:53:26 +0900 Subject: [PATCH 74/89] Manually backports STORM-1056 to 0.9.x --- bin/storm | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/bin/storm b/bin/storm index fee548a97d8..2575f7499c8 100755 --- a/bin/storm +++ b/bin/storm @@ -71,6 +71,8 @@ else: if (not os.path.isfile(USER_CONF_DIR + "/storm.yaml")): USER_CONF_DIR = CLUSTER_CONF_DIR +STORM_SUPERVISOR_LOG_FILE = os.getenv('STORM_SUPERVISOR_LOG_FILE', "supervisor.log") + init_storm_env() CONFIG_OPTS = [] @@ -326,7 +328,7 @@ def supervisor(klass="backtype.storm.daemon.supervisor"): """ cppaths = [CLUSTER_CONF_DIR] jvmopts = parse_args(confvalue("supervisor.childopts", cppaths)) + [ - "-Dlogfile.name=supervisor.log", + "-Dlogfile.name=" + STORM_SUPERVISOR_LOG_FILE, "-Dlogback.configurationFile=" + STORM_DIR + "/logback/cluster.xml", ] exec_storm_class( From 394a4a89e29e703847a30b1348713f0d60ffb0c9 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 24 Sep 2015 09:55:29 +0900 Subject: [PATCH 75/89] add STORM-1056 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 03a677fdae8..570b13c010f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.6 + * STORM-1056: allow supervisor log filename to be configurable via ENV variable * STORM-1051: Netty Client.java's flushMessages produces a NullPointerException * STORM-763: nimbus reassigned worker A to another machine, but other worker's netty client can't connect to the new worker A * STORM-935: Update Disruptor queue version to 2.10.4 From 697059e958879c4daae5f4db57ce9abc04e81bd7 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 15 Oct 2015 16:35:19 -0400 Subject: [PATCH 76/89] [maven-release-plugin] prepare release v0.9.6 --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index f2aacd046b4..408cd21ac7a 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index fe196881254..da6c00df67f 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index a2654900785..9e48789a9aa 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 36f77c33df8..db8375b18f0 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/pom.xml b/pom.xml index 105838d8f5a..bc0bd8f1da0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6-SNAPSHOT + 0.9.6 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.6 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index dfca2a4ae23..e80b4e87674 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 3b7d17717c9..31ac3435b3d 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index bf6dcbf521e..2ea04362ee3 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 87b65a3860c..4ea09046aac 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm From 19a4e129edcaeb8b0615ac928f3107a846992480 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 15 Oct 2015 16:35:27 -0400 Subject: [PATCH 77/89] [maven-release-plugin] prepare for next development iteration --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 408cd21ac7a..96442b1cc5d 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index da6c00df67f..6a68a17dc41 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 9e48789a9aa..5a97fc24957 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index db8375b18f0..4f9bf4b9a58 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index bc0bd8f1da0..9676f868dbe 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6 + 0.9.7-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.6 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e80b4e87674..735301b8194 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 31ac3435b3d..d21a6345e36 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 2ea04362ee3..9aa84ac3d20 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 4ea09046aac..a468153628e 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml org.apache.storm From 8a29593539869e9144af45e8da11bf719f435d05 Mon Sep 17 00:00:00 2001 From: Kyle Nusbaum Date: Tue, 1 Sep 2015 12:13:05 -0500 Subject: [PATCH 78/89] Netty Fix. --- .../storm/messaging/netty/Client.java | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 067dddc4f12..56ed300889e 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -119,6 +119,8 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private final MessageBuffer batcher; + private final Object writeLock = new Object(); + @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, HashedWheelTimer scheduler, String host, int port) { closing = false; @@ -240,21 +242,24 @@ public void send(Iterator msgs) { return; } - - while (msgs.hasNext()) { - TaskMessage message = msgs.next(); - MessageBatch full = batcher.add(message); - if(full != null){ - flushMessages(channel, full); + synchronized (writeLock) { + while (msgs.hasNext()) { + TaskMessage message = msgs.next(); + MessageBatch full = batcher.add(message); + if(full != null){ + flushMessages(channel, full); + } } } - + if(channel.isWritable()){ - // Netty's internal buffer is not full and we still have message left in the buffer. - // We should write the unfilled MessageBatch immediately to reduce latency - MessageBatch batch = batcher.drain(); - if(batch != null) { - flushMessages(channel, batch); + synchronized (writeLock) { + // Netty's internal buffer is not full and we still have message left in the buffer. + // We should write the unfilled MessageBatch immediately to reduce latency + MessageBatch batch = batcher.drain(); + if(batch != null) { + flushMessages(channel, batch); + } } } else { // Channel's buffer is full, meaning that we have time to wait other messages to arrive, and create a bigger @@ -434,9 +439,11 @@ public String toString() { */ public void notifyInterestChanged(Channel channel) { if(channel.isWritable()){ - // Channel is writable again, write if there are any messages pending - MessageBatch pending = batcher.drain(); - flushMessages(channel, pending); + synchronized (writeLock) { + // Channel is writable again, write if there are any messages pending + MessageBatch pending = batcher.drain(); + flushMessages(channel, pending); + } } } From 5ce3f11f3184cd27b62492cffc28b6c32a7b8069 Mon Sep 17 00:00:00 2001 From: Kyle Nusbaum Date: Tue, 1 Sep 2015 12:37:01 -0500 Subject: [PATCH 79/89] Whitespace --- .../src/jvm/backtype/storm/messaging/netty/Client.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java index 56ed300889e..0bb28fd9bcb 100644 --- a/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java +++ b/storm-core/src/jvm/backtype/storm/messaging/netty/Client.java @@ -120,7 +120,7 @@ public class Client extends ConnectionWithStatus implements IStatefulObject { private final MessageBuffer batcher; private final Object writeLock = new Object(); - + @SuppressWarnings("rawtypes") Client(Map stormConf, ChannelFactory factory, HashedWheelTimer scheduler, String host, int port) { closing = false; @@ -251,7 +251,7 @@ public void send(Iterator msgs) { } } } - + if(channel.isWritable()){ synchronized (writeLock) { // Netty's internal buffer is not full and we still have message left in the buffer. @@ -315,7 +315,7 @@ private void flushMessages(Channel channel, final MessageBatch batch) { if (null == batch || batch.isEmpty()) { return; } - + final int numMessages = batch.size(); LOG.debug("writing {} messages to channel {}", batch.size(), channel.toString()); pendingMessages.addAndGet(numMessages); From ed2adde71ebbf46798c85114dad397e5257ce7bf Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Sat, 17 Oct 2015 23:18:51 +0900 Subject: [PATCH 80/89] add STORM-996 to CHANGELOG.md --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 570b13c010f..cd400780dca 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.6 + * STORM-996: netty-unit-tests/test-batch demonstrates out-of-order delivery * STORM-1056: allow supervisor log filename to be configurable via ENV variable * STORM-1051: Netty Client.java's flushMessages produces a NullPointerException * STORM-763: nimbus reassigned worker A to another machine, but other worker's netty client can't connect to the new worker A From 3fc72fbca91fda1f85ae9b8823001152047b6763 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 22 Oct 2015 16:37:59 -0400 Subject: [PATCH 81/89] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 19a4e129edcaeb8b0615ac928f3107a846992480. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 96442b1cc5d..408cd21ac7a 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index 6a68a17dc41..da6c00df67f 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 5a97fc24957..9e48789a9aa 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 4f9bf4b9a58..db8375b18f0 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/pom.xml b/pom.xml index 9676f868dbe..bc0bd8f1da0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.7-SNAPSHOT + 0.9.6 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.6 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 735301b8194..e80b4e87674 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index d21a6345e36..31ac3435b3d 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 9aa84ac3d20..2ea04362ee3 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index a468153628e..4ea09046aac 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm From fbf98188a2efa3249d246f4c40a8779fe555dc4d Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 22 Oct 2015 16:38:06 -0400 Subject: [PATCH 82/89] Revert "[maven-release-plugin] prepare release v0.9.6" This reverts commit 697059e958879c4daae5f4db57ce9abc04e81bd7. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 408cd21ac7a..f2aacd046b4 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index da6c00df67f..fe196881254 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 9e48789a9aa..a2654900785 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index db8375b18f0..36f77c33df8 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index bc0bd8f1da0..105838d8f5a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6 + 0.9.6-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.6 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e80b4e87674..dfca2a4ae23 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 31ac3435b3d..3b7d17717c9 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 2ea04362ee3..bf6dcbf521e 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 4ea09046aac..87b65a3860c 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm From 39418b20e4c2d03e133d028507bb2f8961861cf0 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 22 Oct 2015 16:54:46 -0400 Subject: [PATCH 83/89] [maven-release-plugin] prepare release v0.9.6 --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index f2aacd046b4..408cd21ac7a 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index fe196881254..da6c00df67f 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index a2654900785..9e48789a9aa 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 36f77c33df8..db8375b18f0 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/pom.xml b/pom.xml index 105838d8f5a..bc0bd8f1da0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6-SNAPSHOT + 0.9.6 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.6 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index dfca2a4ae23..e80b4e87674 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 3b7d17717c9..31ac3435b3d 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index bf6dcbf521e..2ea04362ee3 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 87b65a3860c..4ea09046aac 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm From 45e61777b11ea03014f9944becca1fe511e84fb6 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Thu, 22 Oct 2015 16:54:54 -0400 Subject: [PATCH 84/89] [maven-release-plugin] prepare for next development iteration --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 408cd21ac7a..96442b1cc5d 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index da6c00df67f..6a68a17dc41 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 9e48789a9aa..5a97fc24957 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index db8375b18f0..4f9bf4b9a58 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index bc0bd8f1da0..9676f868dbe 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6 + 0.9.7-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.6 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e80b4e87674..735301b8194 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 31ac3435b3d..d21a6345e36 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 2ea04362ee3..9aa84ac3d20 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 4ea09046aac..a468153628e 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml org.apache.storm From 266e0fe8e33dbb81144c606ae65cf8af5fc6488d Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 28 Oct 2015 16:22:22 -0400 Subject: [PATCH 85/89] add STORM-1027 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index cd400780dca..7fb33137543 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 0.9.6 + * STORM-1027: Use overflow buffer for emitting metrics * STORM-996: netty-unit-tests/test-batch demonstrates out-of-order delivery * STORM-1056: allow supervisor log filename to be configurable via ENV variable * STORM-1051: Netty Client.java's flushMessages produces a NullPointerException From 1b6865208c3812a71d08dbbc6fe5a37c763be834 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 28 Oct 2015 16:22:44 -0400 Subject: [PATCH 86/89] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 45e61777b11ea03014f9944becca1fe511e84fb6. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 96442b1cc5d..408cd21ac7a 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index 6a68a17dc41..da6c00df67f 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 5a97fc24957..9e48789a9aa 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 4f9bf4b9a58..db8375b18f0 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/pom.xml b/pom.xml index 9676f868dbe..bc0bd8f1da0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.7-SNAPSHOT + 0.9.6 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.6 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index 735301b8194..e80b4e87674 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index d21a6345e36..31ac3435b3d 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 9aa84ac3d20..2ea04362ee3 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index a468153628e..4ea09046aac 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.7-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm From 2c4e01783fdb3bdece07c1b1c368979af54fce2c Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 28 Oct 2015 16:22:48 -0400 Subject: [PATCH 87/89] Revert "[maven-release-plugin] prepare release v0.9.6" This reverts commit 39418b20e4c2d03e133d028507bb2f8961861cf0. --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 408cd21ac7a..f2aacd046b4 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index da6c00df67f..fe196881254 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 9e48789a9aa..a2654900785 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index db8375b18f0..36f77c33df8 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index bc0bd8f1da0..105838d8f5a 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6 + 0.9.6-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.6 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e80b4e87674..dfca2a4ae23 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 31ac3435b3d..3b7d17717c9 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 2ea04362ee3..bf6dcbf521e 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 4ea09046aac..87b65a3860c 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.6-SNAPSHOT ../../pom.xml org.apache.storm From 90a97ad6101de3c01233384e3dd3eeff2dde2ba3 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 28 Oct 2015 16:31:14 -0400 Subject: [PATCH 88/89] [maven-release-plugin] prepare release v0.9.6 --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index f2aacd046b4..408cd21ac7a 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index fe196881254..da6c00df67f 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index a2654900785..9e48789a9aa 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index 36f77c33df8..db8375b18f0 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/pom.xml b/pom.xml index 105838d8f5a..bc0bd8f1da0 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6-SNAPSHOT + 0.9.6 pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - HEAD + v0.9.6 https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index dfca2a4ae23..e80b4e87674 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 3b7d17717c9..31ac3435b3d 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index bf6dcbf521e..2ea04362ee3 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 87b65a3860c..4ea09046aac 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6-SNAPSHOT + 0.9.6 ../../pom.xml org.apache.storm From e7c0f239bdd1bb85d8f95084feb617bc10041cd4 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 28 Oct 2015 16:31:26 -0400 Subject: [PATCH 89/89] [maven-release-plugin] prepare for next development iteration --- examples/storm-starter/pom.xml | 2 +- external/storm-hbase/pom.xml | 2 +- external/storm-hdfs/pom.xml | 2 +- external/storm-kafka/pom.xml | 2 +- pom.xml | 4 ++-- storm-buildtools/maven-shade-clojure-transformer/pom.xml | 2 +- storm-core/pom.xml | 2 +- storm-dist/binary/pom.xml | 2 +- storm-dist/source/pom.xml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml index 408cd21ac7a..96442b1cc5d 100644 --- a/examples/storm-starter/pom.xml +++ b/examples/storm-starter/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml index da6c00df67f..6a68a17dc41 100644 --- a/external/storm-hbase/pom.xml +++ b/external/storm-hbase/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml index 9e48789a9aa..5a97fc24957 100644 --- a/external/storm-hdfs/pom.xml +++ b/external/storm-hdfs/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml index db8375b18f0..4f9bf4b9a58 100644 --- a/external/storm-kafka/pom.xml +++ b/external/storm-kafka/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/pom.xml b/pom.xml index bc0bd8f1da0..9676f868dbe 100644 --- a/pom.xml +++ b/pom.xml @@ -27,7 +27,7 @@ org.apache.storm storm - 0.9.6 + 0.9.7-SNAPSHOT pom Storm Distributed and fault-tolerant realtime computation @@ -166,7 +166,7 @@ scm:git:https://git-wip-us.apache.org/repos/asf/storm.git scm:git:https://git-wip-us.apache.org/repos/asf/storm.git - v0.9.6 + HEAD https://git-wip-us.apache.org/repos/asf/storm diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml index e80b4e87674..735301b8194 100644 --- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml +++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml diff --git a/storm-core/pom.xml b/storm-core/pom.xml index 31ac3435b3d..d21a6345e36 100644 --- a/storm-core/pom.xml +++ b/storm-core/pom.xml @@ -20,7 +20,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT org.apache.storm storm-core diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml index 2ea04362ee3..9aa84ac3d20 100644 --- a/storm-dist/binary/pom.xml +++ b/storm-dist/binary/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml org.apache.storm diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml index 4ea09046aac..a468153628e 100644 --- a/storm-dist/source/pom.xml +++ b/storm-dist/source/pom.xml @@ -21,7 +21,7 @@ storm org.apache.storm - 0.9.6 + 0.9.7-SNAPSHOT ../../pom.xml org.apache.storm