You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2014/10/15 18:15:14 UTC

[01/50] git commit: Do not count bolt acks & fails in total stats

Repository: storm
Updated Branches:
  refs/heads/security 4788c7ece -> ea0b8a2a5


Do not count bolt acks & fails in total stats


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/985fb1f5
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/985fb1f5
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/985fb1f5

Branch: refs/heads/security
Commit: 985fb1f531de086a9b1d75df6fb05945a1517230
Parents: cfcedcf
Author: Derek Dagit <de...@yahoo-inc.com>
Authored: Wed Sep 24 00:50:30 2014 -0500
Committer: Derek Dagit <de...@yahoo-inc.com>
Committed: Wed Sep 24 00:50:30 2014 -0500

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/ui/core.clj | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/985fb1f5/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/ui/core.clj b/storm-core/src/clj/backtype/storm/ui/core.clj
index 9937607..1f82411 100644
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@ -341,10 +341,11 @@
         (merge-with + s1 s2))
       (select-keys
         agg-bolt-stats
-        [:emitted :transferred :acked :failed :complete-latencies])
-      (select-keys
-        agg-spout-stats
-        [:emitted :transferred :acked :failed :complete-latencies]))))
+        ;; Include only keys that will be used.  We want to count acked and
+        ;; failed only for the "tuple trees," so we do not include those keys
+        ;; from the bolt executors.
+        [:emitted :transferred])
+      agg-spout-stats)))
 
 (defn stats-times
   [stats-map]


[28/50] git commit: Merge branch 'shade-remove-provided' of https://github.com/ptgoetz/incubator-storm into STORM-499 STORM-499: Document and clean up shaded dependncy resolution with maven

Posted by bo...@apache.org.
Merge branch 'shade-remove-provided' of https://github.com/ptgoetz/incubator-storm into STORM-499
STORM-499: Document and clean up shaded dependncy resolution with maven


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/be66015d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/be66015d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/be66015d

Branch: refs/heads/security
Commit: be66015dfbfad20451b66db35efb16fef50878c9
Parents: 5aaea84 3265508
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Sat Oct 4 09:33:20 2014 -0700
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Sat Oct 4 09:33:20 2014 -0700

----------------------------------------------------------------------
 storm-core/pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/be66015d/storm-core/pom.xml
----------------------------------------------------------------------


[26/50] git commit: Add STORM-514 to changelog

Posted by bo...@apache.org.
Add STORM-514 to changelog


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/5aaea846
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/5aaea846
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/5aaea846

Branch: refs/heads/security
Commit: 5aaea8460878d3ad75cf1769381e5ab4052ec319
Parents: 9f682eb
Author: Michael G. Noll <mn...@verisign.com>
Authored: Thu Oct 2 09:42:35 2014 +0200
Committer: Michael G. Noll <mn...@verisign.com>
Committed: Thu Oct 2 09:42:35 2014 +0200

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5aaea846/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index cbf4a38..9b28492 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -82,6 +82,7 @@
  * STORM-498: make ZK connection timeout configurable in Kafka spout
  * STORM-428: extracted ITuple interface
  * STORM-508: Update DEVELOPER.md now that Storm has graduated from Incubator
+ * STORM-514: Update storm-starter README now that Storm has graduated from Incubator
 
 ## 0.9.2-incubating
  * STORM-66: send taskid on initial handshake


[45/50] git commit: fix platform-specific path separators (missed in a previous merge)

Posted by bo...@apache.org.
fix platform-specific path separators (missed in a previous merge)


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/7ab4f241
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/7ab4f241
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/7ab4f241

Branch: refs/heads/security
Commit: 7ab4f2418b99ebec0c27c4042942a26e4f4972be
Parents: d2707d3
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 17:16:17 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 17:16:17 2014 -0400

----------------------------------------------------------------------
 .../test/clj/backtype/storm/supervisor_test.clj     | 16 +++++++---------
 1 file changed, 7 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7ab4f241/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index a61a2ae..ba74d88 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -247,7 +247,7 @@
     (let [mock-port "42"
           mock-storm-id "fake-storm-id"
           mock-worker-id "fake-worker-id"
-          mock-cp "/base:/stormjar.jar"
+          mock-cp (str file-path-separator "base" class-path-separator file-path-separator "stormjar.jar")
           exp-args-fn (fn [opts topo-opts classpath]
                        (concat [(supervisor/java-cmd) "-server"]
                                opts
@@ -255,10 +255,8 @@
                                ["-Djava.library.path="
                                 (str "-Dlogfile.name=worker-" mock-port ".log")
                                 "-Dstorm.home="
-                                "-Dstorm.conf.file="
-                                "-Dstorm.options="
-                                "-Dstorm.log.dir=/logs"
-                                "-Dlogback.configurationFile=/logback/cluster.xml"
+                                (str "-Dstorm.log.dir=" file-path-separator "logs")
+                                (str "-Dlogback.configurationFile=" file-path-separator "logback" file-path-separator "cluster.xml")
                                 (str "-Dstorm.id=" mock-storm-id)
                                 (str "-Dworker.id=" mock-worker-id)
                                 (str "-Dworker.port=" mock-port)
@@ -308,14 +306,14 @@
                                                 [0]
                                                 exp-args))))
       (testing "testing topology.classpath is added to classpath"
-        (let [topo-cp "/any/path"
+        (let [topo-cp (str file-path-separator "any" file-path-separator "path")
               exp-args (exp-args-fn [] [] (add-to-classpath mock-cp [topo-cp]))
               mock-supervisor {:conf {STORM-CLUSTER-MODE :distributed}}]
           (stubbing [read-supervisor-storm-conf {TOPOLOGY-CLASSPATH topo-cp}
                      supervisor-stormdist-root nil
                      supervisor/jlp nil
                      launch-process nil
-                     current-classpath "/base"]
+                     current-classpath (str file-path-separator "base")]
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
@@ -331,7 +329,7 @@
                      supervisor-stormdist-root nil
                      supervisor/jlp nil
                      launch-process nil
-                     current-classpath "/base"]
+                     current-classpath (str file-path-separator "base")]
                     (supervisor/launch-worker mock-supervisor
                                               mock-storm-id
                                               mock-port
@@ -471,4 +469,4 @@
      (validate-launched-once (:launched changed)
                              {"sup1" [3 4]}
                              (get-storm-id (:storm-cluster-state cluster) "topology2"))
-     )))
+     )))
\ No newline at end of file


[18/50] git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/storm

Posted by bo...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/f72dad8b
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/f72dad8b
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/f72dad8b

Branch: refs/heads/security
Commit: f72dad8b2d290c607c7c5a9551ae9f4780268f13
Parents: 39ca6f7 24b5eef
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 15:45:01 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 15:45:01 2014 -0700

----------------------------------------------------------------------
 README.markdown | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[10/50] git commit: STORM-428: fixed TridentKafkaTest

Posted by bo...@apache.org.
STORM-428: fixed TridentKafkaTest

* added changelog entry


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9553b148
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9553b148
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9553b148

Branch: refs/heads/security
Commit: 9553b1486b35b35d5895ded1cc892096aadf3055
Parents: d2ca0b3
Author: wurstmeister <wu...@users.noreply.github.com>
Authored: Thu Sep 25 23:13:58 2014 +0100
Committer: wurstmeister <wu...@users.noreply.github.com>
Committed: Thu Sep 25 23:26:35 2014 +0100

----------------------------------------------------------------------
 CHANGELOG.md                                                | 1 +
 .../storm-kafka/src/test/storm/kafka/TridentKafkaTest.java  | 9 +++------
 2 files changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9553b148/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 01d7be9..605a7da 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -77,6 +77,7 @@
  * STORM-501: Missing StormSubmitter API
  * STORM-493: Workers inherit storm.conf.file/storm.options properties of their supervisor
  * STORM-498: make ZK connection timeout configurable in Kafka spout
+ * STORM-428: extracted ITuple interface
 
 ## 0.9.2-incubating
  * STORM-66: send taskid on initial handshake

http://git-wip-us.apache.org/repos/asf/storm/blob/9553b148/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java
----------------------------------------------------------------------
diff --git a/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java b/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java
index 064d598..d8a5e24 100644
--- a/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java
+++ b/external/storm-kafka/src/test/storm/kafka/TridentKafkaTest.java
@@ -18,7 +18,7 @@
 package storm.kafka;
 
 import backtype.storm.Config;
-import com.google.common.collect.Lists;
+import backtype.storm.tuple.Fields;
 import kafka.javaapi.consumer.SimpleConsumer;
 import org.junit.After;
 import org.junit.Before;
@@ -28,8 +28,8 @@ import storm.kafka.trident.mapper.FieldNameBasedTupleToKafkaMapper;
 import storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
 import storm.kafka.trident.selector.DefaultTopicSelector;
 import storm.kafka.trident.selector.KafkaTopicSelector;
-import storm.trident.testing.MockTridentTuple;
 import storm.trident.tuple.TridentTuple;
+import storm.trident.tuple.TridentTupleView;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -38,7 +38,6 @@ public class TridentKafkaTest {
     private KafkaTestBroker broker;
     private TridentKafkaState state;
     private Config config;
-    private KafkaConfig kafkaConfig;
     private SimpleConsumer simpleConsumer;
     private TridentTupleToKafkaMapper mapper;
     private KafkaTopicSelector topicSelector;
@@ -73,10 +72,8 @@ public class TridentKafkaTest {
 
     private List<TridentTuple> generateTupleBatch(String key, String message, int batchsize) {
         List<TridentTuple> batch = new ArrayList<TridentTuple>();
-        List<String> values = Lists.newArrayList(key, message);
-        List<String> fields = Lists.newArrayList("key", "message");
         for(int i =0 ; i < batchsize; i++) {
-            batch.add(new MockTridentTuple(fields, values));
+            batch.add(TridentTupleView.createFreshTuple(new Fields("key", "message"), key, message));
         }
         return batch;
     }


[05/50] git commit: STORM-493: Workers inherit storm.conf.file/storm.options properties of their supervisor

Posted by bo...@apache.org.
STORM-493: Workers inherit storm.conf.file/storm.options properties of their supervisor


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/7f29b73b
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/7f29b73b
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/7f29b73b

Branch: refs/heads/security
Commit: 7f29b73b477a4f5e12958ccbdf5b6724395c7d5b
Parents: b0974a3
Author: Christophe Carré <ch...@ymail.com>
Authored: Thu Sep 11 18:08:17 2014 +0200
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Sep 24 15:53:13 2014 -0500

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/daemon/supervisor.clj | 4 ++++
 storm-core/test/clj/backtype/storm/supervisor_test.clj  | 2 ++
 2 files changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7f29b73b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 9c6e3f8..8c18e1c 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -502,6 +502,8 @@
     :distributed [supervisor storm-id port worker-id]
     (let [conf (:conf supervisor)
           storm-home (System/getProperty "storm.home")
+          storm-options (System/getProperty "storm.options")
+          storm-conf-file (System/getProperty "storm.conf.file")
           storm-log-dir (or (System/getProperty "storm.log.dir") (str storm-home "/logs"))
           stormroot (supervisor-stormdist-root conf storm-id)
           jlp (jlp stormroot conf)
@@ -528,6 +530,8 @@
                     [(str "-Djava.library.path=" jlp)
                      (str "-Dlogfile.name=" logfilename)
                      (str "-Dstorm.home=" storm-home)
+                     (str "-Dstorm.conf.file=" storm-conf-file)
+                     (str "-Dstorm.options=" storm-options)
                      (str "-Dstorm.log.dir=" storm-log-dir)
                      (str "-Dlogback.configurationFile=" storm-home "/logback/cluster.xml")
                      (str "-Dstorm.id=" storm-id)

http://git-wip-us.apache.org/repos/asf/storm/blob/7f29b73b/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/backtype/storm/supervisor_test.clj b/storm-core/test/clj/backtype/storm/supervisor_test.clj
index 5abfa89..a61a2ae 100644
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@ -255,6 +255,8 @@
                                ["-Djava.library.path="
                                 (str "-Dlogfile.name=worker-" mock-port ".log")
                                 "-Dstorm.home="
+                                "-Dstorm.conf.file="
+                                "-Dstorm.options="
                                 "-Dstorm.log.dir=/logs"
                                 "-Dlogback.configurationFile=/logback/cluster.xml"
                                 (str "-Dstorm.id=" mock-storm-id)


[25/50] git commit: STORM-514: Refer to post-graduation Storm website and git repos

Posted by bo...@apache.org.
STORM-514: Refer to post-graduation Storm website and git repos

Also, we highlight even more that users must `cd` into the storm-starter
sub-directory.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9f682eb3
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9f682eb3
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9f682eb3

Branch: refs/heads/security
Commit: 9f682eb3c024d3742e905078c4065b0cbfadf163
Parents: 38367b0
Author: Michael G. Noll <mn...@verisign.com>
Authored: Thu Oct 2 09:24:03 2014 +0200
Committer: Michael G. Noll <mn...@verisign.com>
Committed: Thu Oct 2 09:32:01 2014 +0200

----------------------------------------------------------------------
 examples/storm-starter/README.markdown | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9f682eb3/examples/storm-starter/README.markdown
----------------------------------------------------------------------
diff --git a/examples/storm-starter/README.markdown b/examples/storm-starter/README.markdown
index 276de99..8161a06 100644
--- a/examples/storm-starter/README.markdown
+++ b/examples/storm-starter/README.markdown
@@ -26,7 +26,7 @@ Next, make sure you have the storm-starter code available on your machine.  Git/
 following command to download the latest storm-starter code and change to the new directory that contains the downloaded
 code.
 
-    $ git clone git://github.com/apache/incubator-storm.git && cd incubator-storm/examples/storm-starter
+    $ git clone git://github.com/apache/storm.git && cd storm/examples/storm-starter
 
 
 ## storm-starter overview
@@ -45,7 +45,7 @@ After you have familiarized yourself with these topologies, take a look at the o
 for more advanced implementations.
 
 If you want to learn more about how Storm works, please head over to the
-[Storm project page](http://storm.incubator.apache.org).
+[Storm project page](http://storm.apache.org).
 
 
 <a name="maven"></a>
@@ -74,8 +74,7 @@ of Storm in this local Maven repository at `$HOME/.m2/repository`.
 
 ## Running topologies with Maven
 
-Note: All following examples require that you run `cd examples/storm-starter`
-beforehand.
+> Note: All following examples require that you run `cd examples/storm-starter` beforehand.
 
 storm-starter topologies can be run with the maven-exec-plugin. For example, to
 compile and run `WordCountTopology` in local mode, use the command:
@@ -143,7 +142,7 @@ The following instructions will import storm-starter as a new project in Intelli
 
 
 * Open _File > Import Project..._ and navigate to the storm-starter directory of your storm clone (e.g.
-  `~/git/incubator-storm/examples/storm-starter`).
+  `~/git/storm/examples/storm-starter`).
 * Select _Import project from external model_, select "Maven", and click _Next_.
 * In the following screen, enable the checkbox _Import Maven projects automatically_.  Leave all other values at their
   defaults.  Click _Next_.


[44/50] git commit: add non-symlink version of storm.js

Posted by bo...@apache.org.
add non-symlink version of storm.js


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/d2707d3c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/d2707d3c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/d2707d3c

Branch: refs/heads/security
Commit: d2707d3c37d9191ca0754243c64099ddd4a07422
Parents: e9953af
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:51:55 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:51:55 2014 -0400

----------------------------------------------------------------------
 storm-core/src/dev/resources/storm.js | 349 +++++++++++++++++++++++++++++
 1 file changed, 349 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d2707d3c/storm-core/src/dev/resources/storm.js
----------------------------------------------------------------------
diff --git a/storm-core/src/dev/resources/storm.js b/storm-core/src/dev/resources/storm.js
new file mode 100755
index 0000000..5c78072
--- /dev/null
+++ b/storm-core/src/dev/resources/storm.js
@@ -0,0 +1,349 @@
+/**
+ * Base classes in node-js for storm Bolt and Spout.
+ * Implements the storm multilang protocol for nodejs.
+ */
+
+
+var fs = require('fs');
+
+function Storm() {
+    this.messagePart = "";
+    this.taskIdsCallbacks = [];
+    this.isFirstMessage = true;
+    this.separator = '\nend\n';
+}
+
+Storm.prototype.sendMsgToParent = function(msg) {
+    var str = JSON.stringify(msg);
+    process.stdout.write(str + this.separator);
+}
+
+Storm.prototype.sync = function() {
+    this.sendMsgToParent({"command":"sync"});
+}
+
+Storm.prototype.sendPid = function(heartbeatdir) {
+    var pid = process.pid;
+    fs.closeSync(fs.openSync(heartbeatdir + "/" + pid, "w"));
+    this.sendMsgToParent({"pid": pid})
+}
+
+Storm.prototype.log = function(msg) {
+    this.sendMsgToParent({"command": "log", "msg": msg});
+}
+
+Storm.prototype.initSetupInfo = function(setupInfo) {
+    var self = this;
+    var callback = function() {
+        self.sendPid(setupInfo['pidDir']);
+    }
+    this.initialize(setupInfo['conf'], setupInfo['context'], callback);
+}
+
+Storm.prototype.startReadingInput = function() {
+    var self = this;
+    process.stdin.on('readable', function() {
+        var chunk = process.stdin.read();
+        var messages = self.handleNewChunk(chunk);
+        messages.forEach(function(message) {
+            self.handleNewMessage(message);
+        })
+
+    });
+}
+
+/**
+ * receives a new string chunk and returns a list of new messages with the separator removed
+ * stores state in this.messagePart
+ * @param chunk
+ */
+Storm.prototype.handleNewChunk = function(chunk) {
+    //invariant: this.messagePart has no separator otherwise we would have parsed it already
+    var messages = [];
+    if (chunk && chunk.length !== 0) {
+        //"{}".split("\nend\n")           ==> ['{}']
+        //"\nend\n".split("\nend\n")      ==> [''  , '']
+        //"{}\nend\n".split("\nend\n")    ==> ['{}', '']
+        //"\nend\n{}".split("\nend\n")    ==> [''  , '{}']
+        // "{}\nend\n{}".split("\nend\n") ==> ['{}', '{}' ]
+        this.messagePart = this.messagePart + chunk;
+        var newMessageParts = this.messagePart.split(this.separator);
+        while (newMessageParts.length > 0) {
+            var potentialMessage = newMessageParts.shift();
+            var anotherMessageAhead = newMessageParts.length > 0;
+            if  (!anotherMessageAhead) {
+                this.messagePart = potentialMessage;
+            }
+            else if (potentialMessage.length > 0) {
+                messages.push(potentialMessage);
+            }
+        }
+    }
+    return messages;
+ }
+
+Storm.prototype.isTaskIds = function(msg) {
+    return (msg instanceof Array);
+}
+
+Storm.prototype.handleNewMessage = function(msg) {
+    var parsedMsg = JSON.parse(msg);
+
+    if (this.isFirstMessage) {
+        this.initSetupInfo(parsedMsg);
+        this.isFirstMessage = false;
+    } else if (this.isTaskIds(parsedMsg)) {
+        this.handleNewTaskId(parsedMsg);
+    } else {
+        this.handleNewCommand(parsedMsg);
+    }
+}
+
+Storm.prototype.handleNewTaskId = function(taskIds) {
+    //When new list of task ids arrives, the callback that was passed with the corresponding emit should be called.
+    //Storm assures that the task ids will be sent in the same order as their corresponding emits so it we can simply
+    //take the first callback in the list and be sure it is the right one.
+
+    var callback = this.taskIdsCallbacks.shift();
+    if (callback) {
+        callback(taskIds);
+    } else {
+        throw new Error('Something went wrong, we off the split of task id callbacks');
+    }
+}
+
+
+
+/**
+ *
+ * @param messageDetails json with the emit details.
+ *
+ * For bolt, the json must contain the required fields:
+ * - tuple - the value to emit
+ * - anchorTupleId - the value of the anchor tuple (the input tuple that lead to this emit). Used to track the source
+ * tuple and return ack when all components successfully finished to process it.
+ * and may contain the optional fields:
+ * - stream (if empty - emit to default stream)
+ *
+ * For spout, the json must contain the required fields:
+ * - tuple - the value to emit
+ *
+ * and may contain the optional fields:
+ * - id - pass id for reliable emit (and receive ack/fail later).
+ * - stream - if empty - emit to default stream.
+ *
+ * @param onTaskIds function than will be called with list of task ids the message was emitted to (when received).
+ */
+Storm.prototype.emit = function(messageDetails, onTaskIds) {
+    //Every emit triggers a response - list of task ids to which the tuple was emitted. The task ids are accessible
+    //through the callback (will be called when the response arrives). The callback is stored in a list until the
+    //corresponding task id list arrives.
+    if (messageDetails.task) {
+        throw new Error('Illegal input - task. To emit to specific task use emit direct!');
+    }
+
+    if (!onTaskIds) {
+        throw new Error('You must pass a onTaskIds callback when using emit!')
+    }
+
+    this.taskIdsCallbacks.push(onTaskIds);
+    this.__emit(messageDetails);;
+}
+
+
+/**
+ * Emit message to specific task.
+ * @param messageDetails json with the emit details.
+ *
+ * For bolt, the json must contain the required fields:
+ * - tuple - the value to emit
+ * - anchorTupleId - the value of the anchor tuple (the input tuple that lead to this emit). Used to track the source
+ * tuple and return ack when all components successfully finished to process it.
+ * - task - indicate the task to send the tuple to.
+ * and may contain the optional fields:
+ * - stream (if empty - emit to default stream)
+ *
+ * For spout, the json must contain the required fields:
+ * - tuple - the value to emit
+ * - task - indicate the task to send the tuple to.
+ * and may contain the optional fields:
+ * - id - pass id for reliable emit (and receive ack/fail later).
+ * - stream - if empty - emit to default stream.
+ *
+ * @param onTaskIds function than will be called with list of task ids the message was emitted to (when received).
+ */
+Storm.prototype.emitDirect = function(commandDetails) {
+    if (!commandDetails.task) {
+        throw new Error("Emit direct must receive task id!")
+    }
+    this.__emit(commandDetails);
+}
+
+/**
+ * Initialize storm component according to the configuration received.
+ * @param conf configuration object accrding to storm protocol.
+ * @param context context object according to storm protocol.
+ * @param done callback. Call this method when finished initializing.
+ */
+Storm.prototype.initialize = function(conf, context, done) {
+    done();
+}
+
+Storm.prototype.run = function() {
+    process.stdout.setEncoding('utf8');
+    process.stdin.setEncoding('utf8');
+    this.startReadingInput();
+}
+
+function Tuple(id, component, stream, task, values) {
+    this.id = id;
+    this.component = component;
+    this.stream = stream;
+    this.task = task;
+    this.values = values;
+}
+
+/**
+ * Base class for storm bolt.
+ * To create a bolt implement 'process' method.
+ * You may also implement initialize method to
+ */
+function BasicBolt() {
+    Storm.call(this);
+    this.anchorTuple = null;
+};
+
+BasicBolt.prototype = Object.create(Storm.prototype);
+BasicBolt.prototype.constructor = BasicBolt;
+
+/**
+ * Emit message.
+ * @param commandDetails json with the required fields:
+ * - tuple - the value to emit
+ * - anchorTupleId - the value of the anchor tuple (the input tuple that lead to this emit). Used to track the source
+ * tuple and return ack when all components successfully finished to process it.
+ * and the optional fields:
+ * - stream (if empty - emit to default stream)
+ * - task (pass only to emit to specific task)
+ */
+BasicBolt.prototype.__emit = function(commandDetails) {
+    var self = this;
+
+    var message = {
+        command: "emit",
+        tuple: commandDetails.tuple,
+        stream: commandDetails.stream,
+        task: commandDetails.task,
+        anchors: [commandDetails.anchorTupleId]
+    };
+
+    this.sendMsgToParent(message);
+}
+
+BasicBolt.prototype.handleNewCommand = function(command) {
+    var self = this;
+    var tup = new Tuple(command["id"], command["comp"], command["stream"], command["task"], command["tuple"]);
+    var callback = function(err) {
+          if (err) {
+              self.fail(tup, err);
+              return;
+          }
+          self.ack(tup);
+      }
+    this.process(tup, callback);
+}
+
+/**
+ * Implement this method when creating a bolt. This is the main method that provides the logic of the bolt (what
+ * should it do?).
+ * @param tuple the input of the bolt - what to process.
+ * @param done call this method when done processing.
+ */
+BasicBolt.prototype.process = function(tuple, done) {};
+
+BasicBolt.prototype.ack = function(tup) {
+    this.sendMsgToParent({"command": "ack", "id": tup.id});
+}
+
+BasicBolt.prototype.fail = function(tup, err) {
+    this.sendMsgToParent({"command": "fail", "id": tup.id});
+}
+
+
+/**
+ * Base class for storm spout.
+ * To create a spout implement the following methods: nextTuple, ack and fail (nextTuple - mandatory, ack and fail
+ * can stay empty).
+ * You may also implement initialize method.
+ *
+ */
+function Spout() {
+    Storm.call(this);
+};
+
+Spout.prototype = Object.create(Storm.prototype);
+
+Spout.prototype.constructor = Spout;
+
+/**
+ * This method will be called when an ack is received for preciously sent tuple. One may implement it.
+ * @param id The id of the tuple.
+ * @param done Call this method when finished and ready to receive more tuples.
+ */
+Spout.prototype.ack = function(id, done) {};
+
+/**
+ * This method will be called when an fail is received for preciously sent tuple. One may implement it (for example -
+ * log the failure or send the tuple again).
+ * @param id The id of the tuple.
+ * @param done Call this method when finished and ready to receive more tuples.
+ */
+Spout.prototype.fail = function(id, done) {};
+
+/**
+ * Method the indicates its time to emit the next tuple.
+ * @param done call this method when done sending the output.
+ */
+Spout.prototype.nextTuple = function(done) {};
+
+Spout.prototype.handleNewCommand = function(command) {
+    var self = this;
+    var callback = function() {
+        self.sync();
+    }
+
+    if (command["command"] === "next") {
+        this.nextTuple(callback);
+    }
+
+    if (command["command"] === "ack") {
+        this.ack(command["id"], callback);
+    }
+
+    if (command["command"] === "fail") {
+        this.fail(command["id"], callback);
+    }
+}
+
+/**
+ * @param commandDetails json with the required fields:
+ * - tuple - the value to emit.
+ * and the optional fields:
+ * - id - pass id for reliable emit (and receive ack/fail later).
+ * - stream - if empty - emit to default stream.
+ * - task - pass only to emit to specific task.
+ */
+Spout.prototype.__emit = function(commandDetails) {
+    var message = {
+        command: "emit",
+        tuple: commandDetails.tuple,
+        id: commandDetails.id,
+        stream: commandDetails.stream,
+        task: commandDetails.task
+    };
+
+    this.sendMsgToParent(message);
+}
+
+module.exports.BasicBolt = BasicBolt;
+module.exports.Spout = Spout;


[27/50] git commit: STORM-488: Exit with 254 error code if storm CLI is run with unknown command

Posted by bo...@apache.org.
STORM-488: Exit with 254 error code if storm CLI is run with unknown command


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b3528c7d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b3528c7d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b3528c7d

Branch: refs/heads/security
Commit: b3528c7d9bcb5a04bc5bac5f7ea571f0c8ea0984
Parents: 5aaea84
Author: Michael G. Noll <mn...@verisign.com>
Authored: Thu Oct 2 10:02:55 2014 +0200
Committer: Michael G. Noll <mn...@verisign.com>
Committed: Thu Oct 2 10:07:27 2014 +0200

----------------------------------------------------------------------
 bin/storm | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b3528c7d/bin/storm
----------------------------------------------------------------------
diff --git a/bin/storm b/bin/storm
index 2c75e58..fee548a 100755
--- a/bin/storm
+++ b/bin/storm
@@ -467,6 +467,7 @@ def print_usage(command=None):
 def unknown_command(*args):
     print("Unknown command: [storm %s]" % ' '.join(sys.argv[1:]))
     print_usage()
+    sys.exit(254)
 
 COMMANDS = {"jar": jar, "kill": kill, "shell": shell, "nimbus": nimbus, "ui": ui, "logviewer": logviewer,
             "drpc": drpc, "supervisor": supervisor, "localconfvalue": print_localconfvalue,


[06/50] git commit: Merge branch 'STORM-498' of github.com:miguno/storm into STORM-498

Posted by bo...@apache.org.
Merge branch 'STORM-498' of github.com:miguno/storm into STORM-498

Conflicts:
	CHANGELOG.md


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/69545361
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/69545361
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/69545361

Branch: refs/heads/security
Commit: 695453619706edb706df8dcb54561e9d17e837a0
Parents: 46f0588 2596e33
Author: Michael G. Noll <mn...@verisign.com>
Authored: Thu Sep 25 12:08:04 2014 +0200
Committer: Michael G. Noll <mn...@verisign.com>
Committed: Thu Sep 25 12:08:04 2014 +0200

----------------------------------------------------------------------
 CHANGELOG.md                                                       | 1 +
 external/storm-kafka/src/jvm/storm/kafka/DynamicBrokersReader.java | 2 +-
 .../storm-kafka/src/test/storm/kafka/DynamicBrokersReaderTest.java | 1 +
 3 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/69545361/CHANGELOG.md
----------------------------------------------------------------------
diff --cc CHANGELOG.md
index 10b70f1,4af6028..01d7be9
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@@ -74,8 -74,7 +74,9 @@@
   * STORM-336: Logback version should be upgraded
   * STORM-386: nodejs multilang protocol implementation and examples
   * STORM-500: Add Spinner when UI is loading stats from nimbus
 + * STORM-501: Missing StormSubmitter API
 + * STORM-493: Workers inherit storm.conf.file/storm.options properties of their supervisor
+  * STORM-498: make ZK connection timeout configurable in Kafka spout
  
  ## 0.9.2-incubating
   * STORM-66: send taskid on initial handshake


[49/50] git commit: prepare for next development iteration

Posted by bo...@apache.org.
prepare for next development iteration


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/cfe7e631
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/cfe7e631
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/cfe7e631

Branch: refs/heads/security
Commit: cfe7e6313acc35a5c346480b4ac4e0d863d09410
Parents: 3a4274c
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Oct 9 17:22:26 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Oct 9 17:22:26 2014 -0400

----------------------------------------------------------------------
 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                                                  | 2 +-
 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, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/examples/storm-starter/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index 27ea5b7..68fb962 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -20,7 +20,7 @@
   <parent>
       <artifactId>storm</artifactId>
       <groupId>org.apache.storm</groupId>
-      <version>0.9.3-rc1</version>
+      <version>0.9.3-rc2-SNAPSHOT</version>
       <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/external/storm-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml
index da52429..b4bdd6d 100644
--- a/external/storm-hbase/pom.xml
+++ b/external/storm-hbase/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-rc1</version>
+        <version>0.9.3-rc2-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/external/storm-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml
index a2137b5..7afb4a8 100644
--- a/external/storm-hdfs/pom.xml
+++ b/external/storm-hdfs/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-rc1</version>
+        <version>0.9.3-rc2-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/external/storm-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml
index 65a207e..16e994d 100644
--- a/external/storm-kafka/pom.xml
+++ b/external/storm-kafka/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-rc1</version>
+        <version>0.9.3-rc2-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e3e4b7f..b63d332 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,7 +27,7 @@
 
     <groupId>org.apache.storm</groupId>
     <artifactId>storm</artifactId>
-    <version>0.9.3-rc1</version>
+    <version>0.9.3-rc2-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Storm</name>
     <description>Distributed and fault-tolerant realtime computation</description>

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/storm-buildtools/maven-shade-clojure-transformer/pom.xml
----------------------------------------------------------------------
diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml
index beea82f..9e9f524 100644
--- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml
+++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-rc1</version>
+        <version>0.9.3-rc2-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 1b12b7d..d2938e3 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -20,7 +20,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-rc1</version>
+        <version>0.9.3-rc2-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.storm</groupId>
     <artifactId>storm-core</artifactId>

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/storm-dist/binary/pom.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml
index 31c0bf5..f6dd6d5 100644
--- a/storm-dist/binary/pom.xml
+++ b/storm-dist/binary/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-rc1</version>
+        <version>0.9.3-rc2-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <groupId>org.apache.storm</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/cfe7e631/storm-dist/source/pom.xml
----------------------------------------------------------------------
diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml
index deb7bdf..91de1ac 100644
--- a/storm-dist/source/pom.xml
+++ b/storm-dist/source/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-rc1</version>
+        <version>0.9.3-rc2-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <groupId>org.apache.storm</groupId>


[23/50] git commit: update changelog for STORM-210

Posted by bo...@apache.org.
update changelog for STORM-210


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/43a978fd
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/43a978fd
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/43a978fd

Branch: refs/heads/security
Commit: 43a978fde3fb71f93438d0df939c745c186ae67f
Parents: 9f71270
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 16:00:47 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 16:00:47 2014 -0700

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/43a978fd/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index dc7ff8e..cbf4a38 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3-incubating
+ * STORM-210: Add storm-hbase module
  * STORM-507: Topology visualization should not block ui
  * STORM-504: Class used by `repl` command is deprecated.
  * STORM-330: Implement storm exponential backoff stategy for netty client and curator


[24/50] git commit: add Sriharsha Chintalapani as storm-hbase sponsor

Posted by bo...@apache.org.
add Sriharsha Chintalapani as storm-hbase sponsor


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/38367b0d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/38367b0d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/38367b0d

Branch: refs/heads/security
Commit: 38367b0d43ba9a57ecfba3830f3a11414769ca3c
Parents: 43a978f
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 16:07:07 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 16:07:07 2014 -0700

----------------------------------------------------------------------
 external/storm-hbase/README.md | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/38367b0d/external/storm-hbase/README.md
----------------------------------------------------------------------
diff --git a/external/storm-hbase/README.md b/external/storm-hbase/README.md
index b483b1f..a5f252d 100644
--- a/external/storm-hbase/README.md
+++ b/external/storm-hbase/README.md
@@ -210,4 +210,6 @@ under the License.
 
 ## Committer Sponsors
 
- * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
\ No newline at end of file
+ * P. Taylor Goetz ([ptgoetz@apache.org](mailto:ptgoetz@apache.org))
+ * Sriharsha Chintalapani ([sriharsha@apache.org](mailto:sriharsha@apache.org))
+ 
\ No newline at end of file


[47/50] git commit: update changelog for STORM-519

Posted by bo...@apache.org.
update changelog for STORM-519


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/1d9688d7
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/1d9688d7
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/1d9688d7

Branch: refs/heads/security
Commit: 1d9688d710f11e451526fd62109f59e7e63f7b91
Parents: 8c4eebf
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Oct 9 16:25:11 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Oct 9 16:25:11 2014 -0400

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1d9688d7/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 360979a..c8687c6 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3
+ * STORM-519: add tuple as an input param to HBaseValueMapper 
  * STORM-488: Exit with 254 error code if storm CLI is run with unknown command
  * STORM-506: Do not count bolt acks & fails in total stats
  * STORM-490: fix build under Windows


[16/50] git commit: Merge branch 'storm-repl-class' of github.com:strongh/incubator-storm

Posted by bo...@apache.org.
Merge branch 'storm-repl-class' of github.com:strongh/incubator-storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/f3421fbe
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/f3421fbe
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/f3421fbe

Branch: refs/heads/security
Commit: f3421fbe36a3391b2e621b69f47d9b5d066161cc
Parents: 5d1c7c3 0b457f2
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 15:39:25 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 15:39:25 2014 -0700

----------------------------------------------------------------------
 bin/storm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[42/50] git commit: Merge branch 'nodejs-multilang' of github.com:forter/incubator-storm

Posted by bo...@apache.org.
Merge branch 'nodejs-multilang' of github.com:forter/incubator-storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/533f7bae
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/533f7bae
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/533f7bae

Branch: refs/heads/security
Commit: 533f7baebff24331c983ae9caecfa085968c3d11
Parents: 42a6dbd 3b609c2
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:49:33 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:49:33 2014 -0400

----------------------------------------------------------------------
 storm-core/src/dev/resources/storm.js | 350 +----------------------------
 storm-core/src/multilang/js/storm.js  | 349 ++++++++++++++++++++++++++++
 2 files changed, 350 insertions(+), 349 deletions(-)
----------------------------------------------------------------------



[09/50] git commit: STORM-428: extracted ITuple interface

Posted by bo...@apache.org.
STORM-428: extracted ITuple interface

* removed MockTridentTuple to avoid duplication


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/d2ca0b34
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/d2ca0b34
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/d2ca0b34

Branch: refs/heads/security
Commit: d2ca0b344fe685c2978c55b9792cde1ef8273eda
Parents: 6954536
Author: wurstmeister <wu...@users.noreply.github.com>
Authored: Sat Aug 9 13:03:25 2014 +0100
Committer: wurstmeister <wu...@users.noreply.github.com>
Committed: Thu Sep 25 23:26:17 2014 +0100

----------------------------------------------------------------------
 .../src/jvm/backtype/storm/tuple/ITuple.java    | 136 +++++++++++++++
 .../src/jvm/backtype/storm/tuple/Tuple.java     | 113 +------------
 .../storm/trident/testing/MockTridentTuple.java | 166 -------------------
 .../jvm/storm/trident/tuple/TridentTuple.java   |  46 +----
 .../storm/trident/tuple/TridentTupleView.java   |  35 +++-
 .../jvm/storm/trident/tuple/ValuePointer.java   |   1 -
 .../test/clj/storm/trident/tuple_test.clj       |  28 ++--
 7 files changed, 192 insertions(+), 333 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d2ca0b34/storm-core/src/jvm/backtype/storm/tuple/ITuple.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/tuple/ITuple.java b/storm-core/src/jvm/backtype/storm/tuple/ITuple.java
new file mode 100644
index 0000000..c85848d
--- /dev/null
+++ b/storm-core/src/jvm/backtype/storm/tuple/ITuple.java
@@ -0,0 +1,136 @@
+/**
+ * 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.tuple;
+
+import java.util.List;
+
+public interface ITuple {
+
+    /**
+     * Returns the number of fields in this tuple.
+     */
+    public int size();
+
+    /**
+     * Returns true if this tuple contains the specified name of the field.
+     */
+    public boolean contains(String field);
+
+    /**
+     * Gets the names of the fields in this tuple.
+     */
+    public Fields getFields();
+
+    /**
+     * Returns the position of the specified field in this tuple.
+     */
+    public int fieldIndex(String field);
+
+    /**
+     * Returns a subset of the tuple based on the fields selector.
+     */
+    public List<Object> select(Fields selector);
+
+    /**
+     * Gets the field at position i in the tuple. Returns object since tuples are dynamically typed.
+     */
+    public Object getValue(int i);
+
+    /**
+     * Returns the String at position i in the tuple. If that field is not a String,
+     * you will get a runtime error.
+     */
+    public String getString(int i);
+
+    /**
+     * Returns the Integer at position i in the tuple. If that field is not an Integer,
+     * you will get a runtime error.
+     */
+    public Integer getInteger(int i);
+
+    /**
+     * Returns the Long at position i in the tuple. If that field is not a Long,
+     * you will get a runtime error.
+     */
+    public Long getLong(int i);
+
+    /**
+     * Returns the Boolean at position i in the tuple. If that field is not a Boolean,
+     * you will get a runtime error.
+     */
+    public Boolean getBoolean(int i);
+
+    /**
+     * Returns the Short at position i in the tuple. If that field is not a Short,
+     * you will get a runtime error.
+     */
+    public Short getShort(int i);
+
+    /**
+     * Returns the Byte at position i in the tuple. If that field is not a Byte,
+     * you will get a runtime error.
+     */
+    public Byte getByte(int i);
+
+    /**
+     * Returns the Double at position i in the tuple. If that field is not a Double,
+     * you will get a runtime error.
+     */
+    public Double getDouble(int i);
+
+    /**
+     * Returns the Float at position i in the tuple. If that field is not a Float,
+     * you will get a runtime error.
+     */
+    public Float getFloat(int i);
+
+    /**
+     * Returns the byte array at position i in the tuple. If that field is not a byte array,
+     * you will get a runtime error.
+     */
+    public byte[] getBinary(int i);
+
+
+    public Object getValueByField(String field);
+
+    public String getStringByField(String field);
+
+    public Integer getIntegerByField(String field);
+
+    public Long getLongByField(String field);
+
+    public Boolean getBooleanByField(String field);
+
+    public Short getShortByField(String field);
+
+    public Byte getByteByField(String field);
+
+    public Double getDoubleByField(String field);
+
+    public Float getFloatByField(String field);
+
+    public byte[] getBinaryByField(String field);
+
+    /**
+     * Gets all the values in this tuple.
+     */
+    public List<Object> getValues();
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/d2ca0b34/storm-core/src/jvm/backtype/storm/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/backtype/storm/tuple/Tuple.java b/storm-core/src/jvm/backtype/storm/tuple/Tuple.java
index 113b300..34dc61a 100644
--- a/storm-core/src/jvm/backtype/storm/tuple/Tuple.java
+++ b/storm-core/src/jvm/backtype/storm/tuple/Tuple.java
@@ -31,120 +31,9 @@ import java.util.List;
  * use another type, you'll need to implement and register a serializer for that type.
  * See {@link http://github.com/nathanmarz/storm/wiki/Serialization} for more info.
  */
-public interface Tuple {
+public interface Tuple extends ITuple{
 
     /**
-     * Returns the number of fields in this tuple.
-     */
-    public int size();
-    
-    /**
-     * Returns the position of the specified field in this tuple.
-     */
-    public int fieldIndex(String field);
-    
-    /**
-     * Returns true if this tuple contains the specified name of the field.
-     */
-    public boolean contains(String field);
-    
-    /**
-     * Gets the field at position i in the tuple. Returns object since tuples are dynamically typed.
-     */
-    public Object getValue(int i);
-
-    /**
-     * Returns the String at position i in the tuple. If that field is not a String, 
-     * you will get a runtime error.
-     */
-    public String getString(int i);
-
-    /**
-     * Returns the Integer at position i in the tuple. If that field is not an Integer, 
-     * you will get a runtime error.
-     */
-    public Integer getInteger(int i);
-
-    /**
-     * Returns the Long at position i in the tuple. If that field is not a Long, 
-     * you will get a runtime error.
-     */
-    public Long getLong(int i);
-
-    /**
-     * Returns the Boolean at position i in the tuple. If that field is not a Boolean, 
-     * you will get a runtime error.
-     */
-    public Boolean getBoolean(int i);
-
-    /**
-     * Returns the Short at position i in the tuple. If that field is not a Short, 
-     * you will get a runtime error.
-     */
-    public Short getShort(int i);
-
-    /**
-     * Returns the Byte at position i in the tuple. If that field is not a Byte, 
-     * you will get a runtime error.
-     */
-    public Byte getByte(int i);
-
-    /**
-     * Returns the Double at position i in the tuple. If that field is not a Double, 
-     * you will get a runtime error.
-     */
-    public Double getDouble(int i);
-
-    /**
-     * Returns the Float at position i in the tuple. If that field is not a Float, 
-     * you will get a runtime error.
-     */
-    public Float getFloat(int i);
-
-    /**
-     * Returns the byte array at position i in the tuple. If that field is not a byte array, 
-     * you will get a runtime error.
-     */
-    public byte[] getBinary(int i);
-    
-    
-    public Object getValueByField(String field);
-
-    public String getStringByField(String field);
-
-    public Integer getIntegerByField(String field);
-
-    public Long getLongByField(String field);
-
-    public Boolean getBooleanByField(String field);
-
-    public Short getShortByField(String field);
-
-    public Byte getByteByField(String field);
-
-    public Double getDoubleByField(String field);
-
-    public Float getFloatByField(String field);
-
-    public byte[] getBinaryByField(String field);
-    
-    /**
-     * Gets all the values in this tuple.
-     */
-    public List<Object> getValues();
-    
-    /**
-     * Gets the names of the fields in this tuple.
-     */
-    public Fields getFields();
-
-    /**
-     * Returns a subset of the tuple based on the fields selector.
-     */
-    public List<Object> select(Fields selector);
-    
-    
-    /**
      * Returns the global stream id (component + stream) of this tuple.
      */
     public GlobalStreamId getSourceGlobalStreamid();

http://git-wip-us.apache.org/repos/asf/storm/blob/d2ca0b34/storm-core/src/jvm/storm/trident/testing/MockTridentTuple.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/testing/MockTridentTuple.java b/storm-core/src/jvm/storm/trident/testing/MockTridentTuple.java
deleted file mode 100644
index b5ee9cb..0000000
--- a/storm-core/src/jvm/storm/trident/testing/MockTridentTuple.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * 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 storm.trident.testing;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import storm.trident.tuple.TridentTuple;
-
-/**
- * A tuple intended for use in testing.
- */
-public class MockTridentTuple extends ArrayList<Object> implements TridentTuple{
-  private final Map<String, Integer> fieldMap;
-
-  public MockTridentTuple(List<String> fieldNames, List<?> values) {
-    super(values);
-    fieldMap = setupFieldMap(fieldNames);
-  }
-
-  public MockTridentTuple(List<String> fieldName, Object... values) {
-    super(Arrays.asList(values));
-    fieldMap = setupFieldMap(fieldName);
-  }
-
-  private Map<String, Integer> setupFieldMap(List<String> fieldNames) {
-    Map<String, Integer> newFieldMap = new HashMap<String, Integer>(fieldNames.size());
-
-    int idx = 0;
-    for (String fieldName : fieldNames) {
-      newFieldMap.put(fieldName, idx++);
-    }
-    return newFieldMap;
-  }
-
-  private int getIndex(String fieldName) {
-    Integer index = fieldMap.get(fieldName);
-    if (index == null) {
-      throw new IllegalArgumentException("Unknown field name: " + fieldName);
-    }
-    return index;
-  }
-
-  @Override
-  public List<Object> getValues() {
-    return this;
-  }
-
-  @Override
-  public Object getValue(int i) {
-    return get(i);
-  }
-
-  @Override
-  public String getString(int i) {
-    return (String)get(i);
-  }
-
-  @Override
-  public Integer getInteger(int i) {
-    return (Integer)get(i);
-  }
-
-  @Override
-  public Long getLong(int i) {
-    return (Long)get(i);
-  }
-
-  @Override
-  public Boolean getBoolean(int i) {
-    return (Boolean)get(i);
-  }
-
-  @Override
-  public Short getShort(int i) {
-    return (Short)get(i);
-  }
-
-  @Override
-  public Byte getByte(int i) {
-    return (Byte)get(i);
-  }
-
-  @Override
-  public Double getDouble(int i) {
-    return (Double)get(i);
-  }
-
-  @Override
-  public Float getFloat(int i) {
-    return (Float)get(i);
-  }
-
-  @Override
-  public byte[] getBinary(int i) {
-    return (byte[]) get(i);
-  }
-
-  @Override
-  public Object getValueByField(String field) {
-    return get(getIndex(field));
-  }
-
-  @Override
-  public String getStringByField(String field) {
-    return (String) getValueByField(field);
-  }
-
-  @Override
-  public Integer getIntegerByField(String field) {
-    return (Integer) getValueByField(field);
-  }
-
-  @Override
-  public Long getLongByField(String field) {
-    return (Long) getValueByField(field);
-  }
-
-  @Override
-  public Boolean getBooleanByField(String field) {
-    return (Boolean) getValueByField(field);
-  }
-
-  @Override
-  public Short getShortByField(String field) {
-    return (Short) getValueByField(field);
-  }
-
-  @Override
-  public Byte getByteByField(String field) {
-    return (Byte) getValueByField(field);
-  }
-
-  @Override
-  public Double getDoubleByField(String field) {
-    return (Double) getValueByField(field);
-  }
-
-  @Override
-  public Float getFloatByField(String field) {
-    return (Float) getValueByField(field);
-  }
-
-  @Override
-  public byte[] getBinaryByField(String field) {
-    return (byte[]) getValueByField(field);
-  }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/d2ca0b34/storm-core/src/jvm/storm/trident/tuple/TridentTuple.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/tuple/TridentTuple.java b/storm-core/src/jvm/storm/trident/tuple/TridentTuple.java
index fbff47c..3dd7e8f 100644
--- a/storm-core/src/jvm/storm/trident/tuple/TridentTuple.java
+++ b/storm-core/src/jvm/storm/trident/tuple/TridentTuple.java
@@ -17,56 +17,18 @@
  */
 package storm.trident.tuple;
 
+import backtype.storm.tuple.ITuple;
+
 import java.io.Serializable;
 import java.util.List;
 import java.util.Map;
 
-public interface TridentTuple extends List<Object> {
+public interface TridentTuple extends ITuple, List<Object> {
+
     public static interface Factory extends Serializable {
         Map<String, ValuePointer> getFieldIndex();
         List<String> getOutputFields();
         int numDelegates();
     }
 
-    List<Object> getValues();
-    
-    Object getValue(int i);
-    
-    String getString(int i);
-    
-    Integer getInteger(int i);
-    
-    Long getLong(int i);
-    
-    Boolean getBoolean(int i);
-    
-    Short getShort(int i);
-    
-    Byte getByte(int i);
-    
-    Double getDouble(int i);
-    
-    Float getFloat(int i);
-    
-    byte[] getBinary(int i);    
-    
-    Object getValueByField(String field);
-    
-    String getStringByField(String field);
-    
-    Integer getIntegerByField(String field);
-    
-    Long getLongByField(String field);
-    
-    Boolean getBooleanByField(String field);
-    
-    Short getShortByField(String field);
-    
-    Byte getByteByField(String field);
-    
-    Double getDoubleByField(String field);
-    
-    Float getFloatByField(String field);
-    
-    byte[] getBinaryByField(String field);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/d2ca0b34/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java b/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java
index 81234d3..0be0c3f 100644
--- a/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java
+++ b/storm-core/src/jvm/storm/trident/tuple/TridentTupleView.java
@@ -29,13 +29,14 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.Arrays;
 
 //extends abstractlist so that it can be emitted directly as Storm tuples
 public class TridentTupleView extends AbstractList<Object> implements TridentTuple {
     ValuePointer[] _index;
     Map<String, ValuePointer> _fieldIndex;
     IPersistentVector _delegates;
-    
+
     public static class ProjectionFactory implements Factory {
         Map<String, ValuePointer> _fieldIndex;
         ValuePointer[] _index;
@@ -199,7 +200,7 @@ public class TridentTupleView extends AbstractList<Object> implements TridentTup
     }
     
     public static TridentTupleView EMPTY_TUPLE = new TridentTupleView(null, new ValuePointer[0], new HashMap());
-    
+
     // index and fieldIndex are precomputed, delegates built up over many operations using persistent data structures
     public TridentTupleView(IPersistentVector delegates, ValuePointer[] index, Map<String, ValuePointer> fieldIndex) {
         _delegates = delegates;
@@ -207,6 +208,16 @@ public class TridentTupleView extends AbstractList<Object> implements TridentTup
         _fieldIndex = fieldIndex;
     }
 
+    public static TridentTuple createFreshTuple(Fields fields, List<Object> values) {
+        FreshOutputFactory factory = new FreshOutputFactory(fields);
+        return factory.create(values);
+    }
+
+    public static TridentTuple createFreshTuple(Fields fields, Object... values) {
+        FreshOutputFactory factory = new FreshOutputFactory(fields);
+        return factory.create(Arrays.asList(values));
+    }
+
     @Override
     public List<Object> getValues() {
         return this;
@@ -218,6 +229,26 @@ public class TridentTupleView extends AbstractList<Object> implements TridentTup
     }
 
     @Override
+    public boolean contains(String field) {
+        return getFields().contains(field);
+    }
+
+    @Override
+    public Fields getFields() {
+        return new Fields(indexToFieldsList(_index));
+    }
+
+    @Override
+    public int fieldIndex(String field) {
+        return getFields().fieldIndex(field);
+    }
+
+    @Override
+    public List<Object> select(Fields selector) {
+        return getFields().select(selector, getValues());
+    }
+
+    @Override
     public Object get(int i) {
         return getValue(i);
     }    

http://git-wip-us.apache.org/repos/asf/storm/blob/d2ca0b34/storm-core/src/jvm/storm/trident/tuple/ValuePointer.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/storm/trident/tuple/ValuePointer.java b/storm-core/src/jvm/storm/trident/tuple/ValuePointer.java
index a510dbe..93470d7 100644
--- a/storm-core/src/jvm/storm/trident/tuple/ValuePointer.java
+++ b/storm-core/src/jvm/storm/trident/tuple/ValuePointer.java
@@ -37,7 +37,6 @@ public class ValuePointer {
             throw new IllegalArgumentException("Fields order must be same length as pointers map");
         }
         ValuePointer[] ret = new ValuePointer[pointers.size()];
-        List<String> flist = fieldsOrder.toList();
         for(int i=0; i<fieldsOrder.size(); i++) {
             ret[i] = pointers.get(fieldsOrder.get(i));
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/d2ca0b34/storm-core/test/clj/storm/trident/tuple_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/storm/trident/tuple_test.clj b/storm-core/test/clj/storm/trident/tuple_test.clj
index 165f165..908e26b 100644
--- a/storm-core/test/clj/storm/trident/tuple_test.clj
+++ b/storm-core/test/clj/storm/trident/tuple_test.clj
@@ -38,12 +38,12 @@
     (bind project-factory (TridentTupleView$ProjectionFactory. fresh-factory (fields "d" "a")))
     (bind tt (.create fresh-factory [3 2 1 4 5]))
     (bind tt2 (.create fresh-factory [9 8 7 6 10]))
-    
+
     (bind pt (.create project-factory tt))
     (bind pt2 (.create project-factory tt2))
     (is (= [4 3] pt))
     (is (= [6 9] pt2))
-    
+
     (is (= 4 (.getValueByField pt "d")))
     (is (= 3 (.getValueByField pt "a")))
     (is (= 6 (.getValueByField pt2 "d")))
@@ -59,7 +59,7 @@
     (bind tt (.create fresh-factory [1 2 3]))
     (bind tt2 (.create append-factory tt [4 5]))
     (bind tt3 (.create append-factory2 tt2 [7]))
-    
+
     (is (= [1 2 3 4 5 7] tt3))
     (is (= 5 (.getValueByField tt2 "e")))
     (is (= 5 (.getValueByField tt3 "e")))
@@ -74,9 +74,9 @@
     (is (= ["a" 1] tt))
     (is (= "a" (.getValueByField tt "a")))
     (is (= 1 (.getValueByField tt "b")))
-    
+
     (bind append-factory (TridentTupleView$OperationOutputFactory. root-factory (fields "c")))
-    
+
     (bind tt2 (.create append-factory tt [3]))
     (is (= ["a" 1 3] tt2))
     (is (= "a" (.getValueByField tt2 "a")))
@@ -91,24 +91,32 @@
     (bind append-factory2 (TridentTupleView$OperationOutputFactory. append-factory1 (fields "e" "f")))
     (bind project-factory1 (TridentTupleView$ProjectionFactory. append-factory2 (fields "a" "f" "b")))
     (bind append-factory3 (TridentTupleView$OperationOutputFactory. project-factory1 (fields "c")))
-  
+
     (bind tt (.create fresh-factory [1 2 3]))
     (bind tt2 (.create append-factory1 tt [4]))
     (bind tt3 (.create append-factory2 tt2 [5 6]))
     (bind tt4 (.create project-factory1 tt3))
     (bind tt5 (.create append-factory3 tt4 [8]))
-  
+
     (is (= [1 2 3] tt))
     (is (= [1 2 3 4] tt2))
     (is (= [1 2 3 4 5 6] tt3))
     (is (= [1 6 2] tt4))
     (is (= [1 6 2 8] tt5))
-  
+
     (is (= 1 (.getValueByField tt5 "a")))
     (is (= 6 (.getValueByField tt5 "f")))
     (is (= 2 (.getValueByField tt5 "b")))
     (is (= 8 (.getValueByField tt5 "c")))
     ))
 
-  
-
+(deftest test-ituple-interface
+  (letlocals
+    (bind tt (TridentTupleView/createFreshTuple (fields "a" "b" "c") [1 2 3]))
+    (is (= [1 2 3] tt))
+    (is (= ["a" "b" "c"] (.toList (.getFields tt))))
+    (is (true? (.contains tt "a")))
+    (is (false? (.contains tt "abcd")))
+    (is (= 0 (.fieldIndex tt "a")))
+    (is (= [3 1] (.select tt (fields "c" "a"))))
+    ))


[07/50] git commit: STORM-508: Update DEVELOPER.md now that Storm has graduated from Incubator

Posted by bo...@apache.org.
STORM-508: Update DEVELOPER.md now that Storm has graduated from Incubator

Storm has recently graduated from Apache Incubator, which resulted in
hyperlink changes to the Storm website (now http://storm.apache.org/)
and to our ASF and GitHub git repositories, respectively.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/70f98f57
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/70f98f57
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/70f98f57

Branch: refs/heads/security
Commit: 70f98f57fa5f2aa9ab720be4712c2170b4e1c2e9
Parents: 6954536
Author: Michael G. Noll <mn...@verisign.com>
Authored: Thu Sep 25 13:13:41 2014 +0200
Committer: Michael G. Noll <mn...@verisign.com>
Committed: Thu Sep 25 13:15:31 2014 +0200

----------------------------------------------------------------------
 DEVELOPER.md | 26 +++++++++++++-------------
 1 file changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/70f98f57/DEVELOPER.md
----------------------------------------------------------------------
diff --git a/DEVELOPER.md b/DEVELOPER.md
index a429d36..81a4122 100644
--- a/DEVELOPER.md
+++ b/DEVELOPER.md
@@ -74,7 +74,7 @@ and summarize the desired functionality.  Set the form field "Issue type" to "Ne
 issue tracker before you will need to register an account (free), log in, and then click on the blue "Create Issue"
 button in the top navigation bar.
 
-You can also opt to send a message to the [Storm Users mailing list](http://storm.incubator.apache.org/community.html).
+You can also opt to send a message to the [Storm Users mailing list](http://storm.apache.org/community.html).
 
 
 <a name="contribute-code"></a>
@@ -83,7 +83,7 @@ You can also opt to send a message to the [Storm Users mailing list](http://stor
 
 Before you set out to contribute code we recommend that you familiarize yourself with the Storm codebase, notably by
 reading through the
-[Implementation documentation](http://storm.incubator.apache.org/documentation/Implementation-docs.html).
+[Implementation documentation](http://storm.apache.org/documentation/Implementation-docs.html).
 
 _If you are interested in contributing code to Storm but do not know where to begin:_
 In this case you should
@@ -124,7 +124,7 @@ GitHub.
 ## Contribute documentation
 
 Documentation contributions are very welcome!  The best way to send contributions is as emails through the
-[Storm Developers](http://storm.incubator.apache.org/community.html) mailing list.
+[Storm Developers](http://storm.apache.org/community.html) mailing list.
 
 
 <a name="pull-requests"></a>
@@ -137,13 +137,13 @@ Documentation contributions are very welcome!  The best way to send contribution
 ### Create a pull request
 
 Pull requests should be done against the read-only git repository at
-[https://github.com/apache/incubator-storm](https://github.com/apache/incubator-storm).
+[https://github.com/apache/storm](https://github.com/apache/storm).
 
 Take a look at [Creating a pull request](https://help.github.com/articles/creating-a-pull-request).  In a nutshell you
 need to:
 
 1. [Fork](https://help.github.com/articles/fork-a-repo) the Storm GitHub repository at
-   [https://github.com/apache/incubator-storm/](https://github.com/apache/incubator-storm/) to your personal GitHub
+   [https://github.com/apache/storm/](https://github.com/apache/storm/) to your personal GitHub
    account.  See [Fork a repo](https://help.github.com/articles/fork-a-repo) for detailed instructions.
 2. Commit any changes to your fork.
 3. Send a [pull request](https://help.github.com/articles/creating-a-pull-request) to the Storm GitHub repository
@@ -152,7 +152,7 @@ need to:
    ticket number (e.g. `STORM-123: ...`).
 
 You may want to read [Syncing a fork](https://help.github.com/articles/syncing-a-fork) for instructions on how to keep
-your fork up to date with the latest changes of the upstream (official) `incubator-storm` repository.
+your fork up to date with the latest changes of the upstream (official) `storm` repository.
 
 
 <a name="approve-pull-request"></a>
@@ -175,11 +175,11 @@ _This section applies to committers only._
 **Important: A pull request must first be properly approved before you are allowed to merge it.**
 
 Committers that are integrating patches or pull requests should use the official Apache repository at
-[https://git-wip-us.apache.org/repos/asf/incubator-storm.git](https://git-wip-us.apache.org/repos/asf/incubator-storm.git).
+[https://git-wip-us.apache.org/repos/asf/storm.git](https://git-wip-us.apache.org/repos/asf/storm.git).
 
 To pull in a merge request you should generally follow the command line instructions sent out by GitHub.
 
-1. Go to your local copy of the [Apache git repo](https://git-wip-us.apache.org/repos/asf/incubator-storm.git), switch
+1. Go to your local copy of the [Apache git repo](https://git-wip-us.apache.org/repos/asf/storm.git), switch
    to the `master` branch, and make sure it is up to date.
 
         $ git checkout master
@@ -297,21 +297,21 @@ The source code of Storm is managed via [git](http://git-scm.com/).  For a numbe
 repository associated with Storm.
 
 * **Committers only:**
-  [https://git-wip-us.apache.org/repos/asf/incubator-storm.git](https://git-wip-us.apache.org/repos/asf/incubator-storm.git)
+  [https://git-wip-us.apache.org/repos/asf/storm.git](https://git-wip-us.apache.org/repos/asf/storm.git)
   is the official and authoritative git repository for Storm, managed under the umbrella of the Apache Software
   Foundation.  Only official Storm committers will interact with this repository.
   When you push the first time to this repository git will prompt you for your username and password.  Use your Apache
   user ID and password, i.e. the credentials you configured via [https://id.apache.org/](https://id.apache.org/) after
   you were [onboarded as a committer](http://www.apache.org/dev/new-committers-guide.html#account-creation).
 * **Everybody else:**
-  [https://github.com/apache/incubator-storm/](https://github.com/apache/incubator-storm/) is a read-only mirror of the
+  [https://github.com/apache/storm/](https://github.com/apache/storm/) is a read-only mirror of the
   official git repository.  If you are not a Storm committer (most people) this is the repository you should work
   against.  See _Development workflow_ above on how you can create a pull request, for instance.
 
 An automated bot (called _[ASF GitHub Bot](https://issues.apache.org/jira/secure/ViewProfile.jspa?name=githubbot)_ in
 [Storm JIRA](https://issues.apache.org/jira/browse/STORM)) runs periodically to merge changes in the
-[official Apache repo](https://git-wip-us.apache.org/repos/asf/incubator-storm.git) to the read-only
-[GitHub mirror repository](https://github.com/apache/incubator-storm/), and to merge comments in GitHub pull requests to
+[official Apache repo](https://git-wip-us.apache.org/repos/asf/storm.git) to the read-only
+[GitHub mirror repository](https://github.com/apache/storm/), and to merge comments in GitHub pull requests to
 the [Storm JIRA](https://issues.apache.org/jira/browse/STORM).
 
 
@@ -335,7 +335,7 @@ If you do not have a JIRA account yet, then you can create one via the link abov
 # Questions?
 
 If you have any questions after reading this document, then please reach out to us via the
-[Storm Developers](http://storm.incubator.apache.org/community.html) mailing list.
+[Storm Developers](http://storm.apache.org/community.html) mailing list.
 
 And of course we also welcome any contributions to improve the information in this document!
 <a name="workflow"></a>


[13/50] git commit: update poms to reflect TLP graduation

Posted by bo...@apache.org.
update poms to reflect TLP graduation


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/c683d45a
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/c683d45a
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/c683d45a

Branch: refs/heads/security
Commit: c683d45ad1d2e6172764ce411fea34044e81c882
Parents: a56ccc7
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Sep 30 10:46:56 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Sep 30 10:46:56 2014 -0700

----------------------------------------------------------------------
 examples/storm-starter/pom.xml                           | 2 +-
 external/storm-hdfs/pom.xml                              | 2 +-
 external/storm-kafka/pom.xml                             | 2 +-
 pom.xml                                                  | 2 +-
 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 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/examples/storm-starter/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index 903c6e7..a31304a 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -20,7 +20,7 @@
   <parent>
       <artifactId>storm</artifactId>
       <groupId>org.apache.storm</groupId>
-      <version>0.9.3-incubating-SNAPSHOT</version>
+      <version>0.9.3-SNAPSHOT</version>
       <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/external/storm-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml
index 2aadac0..27aa5fb 100644
--- a/external/storm-hdfs/pom.xml
+++ b/external/storm-hdfs/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-incubating-SNAPSHOT</version>
+        <version>0.9.3-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/external/storm-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml
index 4972619..05bb117 100644
--- a/external/storm-kafka/pom.xml
+++ b/external/storm-kafka/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-incubating-SNAPSHOT</version>
+        <version>0.9.3-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 111672c..0db0ea4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,7 +27,7 @@
 
     <groupId>org.apache.storm</groupId>
     <artifactId>storm</artifactId>
-    <version>0.9.3-incubating-SNAPSHOT</version>
+    <version>0.9.3-SNAPSHOT</version>
     <packaging>pom</packaging>
     <name>Storm</name>
     <description>Distributed and fault-tolerant realtime computation</description>

http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/storm-buildtools/maven-shade-clojure-transformer/pom.xml
----------------------------------------------------------------------
diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml
index a6fbad1..d789f11 100644
--- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml
+++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-incubating-SNAPSHOT</version>
+        <version>0.9.3-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 4c72498..3164176 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -20,7 +20,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-incubating-SNAPSHOT</version>
+        <version>0.9.3-SNAPSHOT</version>
     </parent>
     <groupId>org.apache.storm</groupId>
     <artifactId>storm-core</artifactId>

http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/storm-dist/binary/pom.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml
index 0d97c0b..5d2ecf3 100644
--- a/storm-dist/binary/pom.xml
+++ b/storm-dist/binary/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-incubating-SNAPSHOT</version>
+        <version>0.9.3-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <groupId>org.apache.storm</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/c683d45a/storm-dist/source/pom.xml
----------------------------------------------------------------------
diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml
index 4fedefb..5284b3b 100644
--- a/storm-dist/source/pom.xml
+++ b/storm-dist/source/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-incubating-SNAPSHOT</version>
+        <version>0.9.3-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <groupId>org.apache.storm</groupId>


[46/50] git commit: Merge branch 'STORM-519' of github.com:Parth-Brahmbhatt/incubator-storm

Posted by bo...@apache.org.
Merge branch 'STORM-519' of github.com:Parth-Brahmbhatt/incubator-storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/8c4eebf2
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/8c4eebf2
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/8c4eebf2

Branch: refs/heads/security
Commit: 8c4eebf27cddfe1dca4d7c3618ffba7bf97d999d
Parents: 7ab4f24 f87bb31
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Oct 9 15:54:19 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Oct 9 15:54:19 2014 -0400

----------------------------------------------------------------------
 external/storm-hbase/pom.xml                                   | 2 +-
 .../main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java | 2 +-
 .../org/apache/storm/hbase/bolt/mapper/HBaseValueMapper.java   | 4 +++-
 .../java/org/apache/storm/hbase/trident/state/HBaseState.java  | 6 ++++--
 .../org/apache/storm/hbase/topology/WordCountValueMapper.java  | 3 ++-
 5 files changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[41/50] git commit: update changelog for STORM-488

Posted by bo...@apache.org.
update changelog for STORM-488


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/42a6dbd0
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/42a6dbd0
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/42a6dbd0

Branch: refs/heads/security
Commit: 42a6dbd0f48e31590485fd46c303b01c4b3279d6
Parents: 9b754ee
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:44:58 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:44:58 2014 -0400

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/42a6dbd0/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 482e057..360979a 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3
+ * STORM-488: Exit with 254 error code if storm CLI is run with unknown command
  * STORM-506: Do not count bolt acks & fails in total stats
  * STORM-490: fix build under Windows
  * STORM-439: Replace purl.js qith jquery URL plugin


[11/50] git commit: Added STORM-508 to changelog

Posted by bo...@apache.org.
Added STORM-508 to changelog


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/a56ccc7f
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/a56ccc7f
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/a56ccc7f

Branch: refs/heads/security
Commit: a56ccc7f5f98b34587c307d4cbdb90c91ce947de
Parents: 70f98f5
Author: Michael G. Noll <mn...@verisign.com>
Authored: Fri Sep 26 16:04:46 2014 +0200
Committer: Michael G. Noll <mn...@verisign.com>
Committed: Fri Sep 26 16:04:46 2014 +0200

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a56ccc7f/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 01d7be9..24950b4 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -77,6 +77,7 @@
  * STORM-501: Missing StormSubmitter API
  * STORM-493: Workers inherit storm.conf.file/storm.options properties of their supervisor
  * STORM-498: make ZK connection timeout configurable in Kafka spout
+ * STORM-508: Update DEVELOPER.md now that Storm has graduated from Incubator
 
 ## 0.9.2-incubating
  * STORM-66: send taskid on initial handshake


[20/50] git commit: Merge branch 'storm-507' of github.com:kishorvpatil/incubator-storm

Posted by bo...@apache.org.
Merge branch 'storm-507' of github.com:kishorvpatil/incubator-storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/a49aebb9
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/a49aebb9
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/a49aebb9

Branch: refs/heads/security
Commit: a49aebb930847c4e5b76228464758ec4125aea63
Parents: 3f2eec5 3c18fd7
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 15:56:43 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 15:56:43 2014 -0700

----------------------------------------------------------------------
 storm-core/src/ui/public/topology.html | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[15/50] git commit: Added @anyatch as contributor for STORM-386

Posted by bo...@apache.org.
Added @anyatch as contributor for STORM-386


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/24b5eefb
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/24b5eefb
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/24b5eefb

Branch: refs/heads/security
Commit: 24b5eefb1553ae37951807502a433f24d3e95ec1
Parents: 5d1c7c3
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Tue Sep 30 13:49:28 2014 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Tue Sep 30 13:49:28 2014 -0500

----------------------------------------------------------------------
 README.markdown | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/24b5eefb/README.markdown
----------------------------------------------------------------------
diff --git a/README.markdown b/README.markdown
index 5fe638e..2f90a4e 100644
--- a/README.markdown
+++ b/README.markdown
@@ -174,6 +174,7 @@ under the License.
 * Ariel Valentin ([@arielvalentin](http://blog.arielvalentin.com/))
 * Dane Hammer ([@danehammer](https://github.com/danehammer))
 * Christophe Carre' ([@chrisz](https://github.com/chrisz))
+* Anya Tchernishov ([@anyatch](https://github.com/anyatch))
 
 ## Acknowledgements
 


[39/50] git commit: update changelog for STORM-506

Posted by bo...@apache.org.
update changelog for STORM-506


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/d50b772f
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/d50b772f
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/d50b772f

Branch: refs/heads/security
Commit: d50b772fb3240c6908699c65c32895791f8c2630
Parents: 9e63e85
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:34:24 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:34:24 2014 -0400

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d50b772f/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index d019262..482e057 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3
+ * STORM-506: Do not count bolt acks & fails in total stats
  * STORM-490: fix build under Windows
  * STORM-439: Replace purl.js qith jquery URL plugin
  * STORM-499: Document and clean up shaded dependncy resolution with maven


[12/50] git commit: remove provided dependencies when shading

Posted by bo...@apache.org.
remove provided dependencies when shading


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3265508b
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3265508b
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3265508b

Branch: refs/heads/security
Commit: 3265508bde30144a18c33b748c37f84d5e31bb9c
Parents: a56ccc7
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Sep 30 09:17:49 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Sep 30 09:17:49 2014 -0700

----------------------------------------------------------------------
 storm-core/pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3265508b/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 4c72498..728323d 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -278,8 +278,8 @@
                     </execution>
                 </executions>
                 <configuration>
-                    <keepDependenciesWithProvidedScope>true</keepDependenciesWithProvidedScope>
-                    <promoteTransitiveDependencies>false</promoteTransitiveDependencies>
+                    <keepDependenciesWithProvidedScope>false</keepDependenciesWithProvidedScope>
+                    <promoteTransitiveDependencies>true</promoteTransitiveDependencies>
                     <createDependencyReducedPom>true</createDependencyReducedPom>
                     <minimizeJar>false</minimizeJar>
                     <artifactSet>


[33/50] git commit: update changelog for STORM-439

Posted by bo...@apache.org.
update changelog for STORM-439


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/88b85e63
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/88b85e63
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/88b85e63

Branch: refs/heads/security
Commit: 88b85e63913ff7b5a63533dd9fa7bedbe8df7f2a
Parents: 53245c2
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 15:25:44 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 15:25:44 2014 -0400

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/88b85e63/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index c2a9a70..8e46c64 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3-incubating
+ * STORM-439: Replace purl.js qith jquery URL plugin
  * STORM-499: Document and clean up shaded dependncy resolution with maven
  * STORM-210: Add storm-hbase module
  * STORM-507: Topology visualization should not block ui


[48/50] git commit: 0.9.3-rc1 release commit

Posted by bo...@apache.org.
0.9.3-rc1 release commit


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3a4274c1
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3a4274c1
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3a4274c1

Branch: refs/heads/security
Commit: 3a4274c15492707cc944162c2380f33eb18c75ac
Parents: 1d9688d
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Oct 9 16:39:11 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Oct 9 16:39:11 2014 -0400

----------------------------------------------------------------------
 CHANGELOG.md                                             | 2 +-
 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                                                  | 2 +-
 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 +-
 10 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index c8687c6..6a21b03 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,4 @@
-## 0.9.3
+## 0.9.3-rc1
  * STORM-519: add tuple as an input param to HBaseValueMapper 
  * STORM-488: Exit with 254 error code if storm CLI is run with unknown command
  * STORM-506: Do not count bolt acks & fails in total stats

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/examples/storm-starter/pom.xml
----------------------------------------------------------------------
diff --git a/examples/storm-starter/pom.xml b/examples/storm-starter/pom.xml
index a31304a..27ea5b7 100644
--- a/examples/storm-starter/pom.xml
+++ b/examples/storm-starter/pom.xml
@@ -20,7 +20,7 @@
   <parent>
       <artifactId>storm</artifactId>
       <groupId>org.apache.storm</groupId>
-      <version>0.9.3-SNAPSHOT</version>
+      <version>0.9.3-rc1</version>
       <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/external/storm-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml
index a492164..da52429 100644
--- a/external/storm-hbase/pom.xml
+++ b/external/storm-hbase/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-SNAPSHOT</version>
+        <version>0.9.3-rc1</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/external/storm-hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/pom.xml b/external/storm-hdfs/pom.xml
index 27aa5fb..a2137b5 100644
--- a/external/storm-hdfs/pom.xml
+++ b/external/storm-hdfs/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-SNAPSHOT</version>
+        <version>0.9.3-rc1</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/external/storm-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-kafka/pom.xml b/external/storm-kafka/pom.xml
index 05bb117..65a207e 100644
--- a/external/storm-kafka/pom.xml
+++ b/external/storm-kafka/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-SNAPSHOT</version>
+        <version>0.9.3-rc1</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 83620ce..e3e4b7f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,7 +27,7 @@
 
     <groupId>org.apache.storm</groupId>
     <artifactId>storm</artifactId>
-    <version>0.9.3-SNAPSHOT</version>
+    <version>0.9.3-rc1</version>
     <packaging>pom</packaging>
     <name>Storm</name>
     <description>Distributed and fault-tolerant realtime computation</description>

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/storm-buildtools/maven-shade-clojure-transformer/pom.xml
----------------------------------------------------------------------
diff --git a/storm-buildtools/maven-shade-clojure-transformer/pom.xml b/storm-buildtools/maven-shade-clojure-transformer/pom.xml
index d789f11..beea82f 100644
--- a/storm-buildtools/maven-shade-clojure-transformer/pom.xml
+++ b/storm-buildtools/maven-shade-clojure-transformer/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-SNAPSHOT</version>
+        <version>0.9.3-rc1</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 29338f0..1b12b7d 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -20,7 +20,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-SNAPSHOT</version>
+        <version>0.9.3-rc1</version>
     </parent>
     <groupId>org.apache.storm</groupId>
     <artifactId>storm-core</artifactId>

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/storm-dist/binary/pom.xml
----------------------------------------------------------------------
diff --git a/storm-dist/binary/pom.xml b/storm-dist/binary/pom.xml
index 5d2ecf3..31c0bf5 100644
--- a/storm-dist/binary/pom.xml
+++ b/storm-dist/binary/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-SNAPSHOT</version>
+        <version>0.9.3-rc1</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <groupId>org.apache.storm</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/3a4274c1/storm-dist/source/pom.xml
----------------------------------------------------------------------
diff --git a/storm-dist/source/pom.xml b/storm-dist/source/pom.xml
index 5284b3b..deb7bdf 100644
--- a/storm-dist/source/pom.xml
+++ b/storm-dist/source/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-SNAPSHOT</version>
+        <version>0.9.3-rc1</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
     <groupId>org.apache.storm</groupId>


[50/50] git commit: Merge branch 'master' into security-upmerge

Posted by bo...@apache.org.
Merge branch 'master' into security-upmerge

Conflicts:
	storm-core/src/clj/backtype/storm/daemon/supervisor.clj
	storm-core/src/clj/backtype/storm/ui/core.clj
	storm-core/src/ui/public/templates/topology-page-template.html
	storm-core/test/clj/backtype/storm/supervisor_test.clj


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/ea0b8a2a
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/ea0b8a2a
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/ea0b8a2a

Branch: refs/heads/security
Commit: ea0b8a2a51332020a569ed7aeb3e0647477af763
Parents: 4788c7e cfe7e63
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Oct 15 11:05:10 2014 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Oct 15 11:05:10 2014 -0500

----------------------------------------------------------------------
 CHANGELOG.md                                    |  16 +-
 DEVELOPER.md                                    |  26 +-
 LICENSE                                         |  20 +-
 README.markdown                                 |   3 +
 bin/storm                                       |   3 +-
 examples/storm-starter/README.markdown          |   9 +-
 examples/storm-starter/pom.xml                  |   2 +-
 external/storm-hbase/LICENSE                    | 202 +++++++++++
 external/storm-hbase/README.md                  | 215 ++++++++++++
 external/storm-hbase/pom.xml                    |  78 +++++
 .../storm/hbase/bolt/AbstractHBaseBolt.java     |  69 ++++
 .../org/apache/storm/hbase/bolt/HBaseBolt.java  |  77 ++++
 .../storm/hbase/bolt/HBaseLookupBolt.java       |  83 +++++
 .../storm/hbase/bolt/mapper/HBaseMapper.java    |  48 +++
 .../bolt/mapper/HBaseProjectionCriteria.java    |  85 +++++
 .../hbase/bolt/mapper/HBaseValueMapper.java     |  43 +++
 .../hbase/bolt/mapper/SimpleHBaseMapper.java    |  90 +++++
 .../apache/storm/hbase/common/ColumnList.java   | 202 +++++++++++
 .../apache/storm/hbase/common/HBaseClient.java  | 135 +++++++
 .../org/apache/storm/hbase/common/IColumn.java  |  30 ++
 .../org/apache/storm/hbase/common/ICounter.java |  29 ++
 .../org/apache/storm/hbase/common/Utils.java    |  69 ++++
 .../storm/hbase/security/HBaseSecurityUtil.java |  52 +++
 .../mapper/SimpleTridentHBaseMapper.java        |  89 +++++
 .../trident/mapper/TridentHBaseMapper.java      |  31 ++
 .../hbase/trident/state/HBaseMapState.java      | 233 +++++++++++++
 .../storm/hbase/trident/state/HBaseQuery.java   |  40 +++
 .../storm/hbase/trident/state/HBaseState.java   | 164 +++++++++
 .../hbase/trident/state/HBaseStateFactory.java  |  40 +++
 .../storm/hbase/trident/state/HBaseUpdater.java |  32 ++
 .../storm/hbase/topology/LookupWordCount.java   |  79 +++++
 .../hbase/topology/PersistentWordCount.java     |  81 +++++
 .../storm/hbase/topology/TotalWordCounter.java  |  70 ++++
 .../storm/hbase/topology/WordCountClient.java   |  57 +++
 .../hbase/topology/WordCountValueMapper.java    |  70 ++++
 .../storm/hbase/topology/WordCounter.java       |  59 ++++
 .../apache/storm/hbase/topology/WordSpout.java  |  88 +++++
 .../storm/hbase/trident/PrintFunction.java      |  40 +++
 .../storm/hbase/trident/WordCountTrident.java   | 104 ++++++
 external/storm-hdfs/pom.xml                     |   2 +-
 external/storm-kafka/pom.xml                    |   2 +-
 .../jvm/storm/kafka/DynamicBrokersReader.java   |   2 +-
 .../storm/kafka/DynamicBrokersReaderTest.java   |   1 +
 .../src/test/storm/kafka/TridentKafkaTest.java  |   9 +-
 pom.xml                                         |  27 +-
 .../maven-shade-clojure-transformer/pom.xml     |   2 +-
 storm-core/pom.xml                              |   6 +-
 .../clj/backtype/storm/daemon/supervisor.clj    |   4 +-
 storm-core/src/clj/backtype/storm/ui/core.clj   |  51 +--
 storm-core/src/dev/resources/storm.py           | 248 ++++++++++++-
 storm-core/src/dev/resources/storm.rb           | 228 +++++++++++-
 .../src/jvm/backtype/storm/tuple/ITuple.java    | 136 ++++++++
 .../src/jvm/backtype/storm/tuple/Tuple.java     | 113 +-----
 .../storm/trident/testing/MockTridentTuple.java | 166 ---------
 .../jvm/storm/trident/tuple/TridentTuple.java   |  46 +--
 .../storm/trident/tuple/TridentTupleView.java   |  35 +-
 .../jvm/storm/trident/tuple/ValuePointer.java   |   1 -
 storm-core/src/multilang/js/storm.js            | 349 +++++++++++++++++++
 storm-core/src/ui/public/component.html         |   8 +-
 storm-core/src/ui/public/js/purl.js             | 267 --------------
 storm-core/src/ui/public/js/script.js           |   3 +-
 storm-core/src/ui/public/js/url.min.js          |   1 +
 storm-core/src/ui/public/js/visualization.js    |   2 +-
 .../templates/component-page-template.html      |   6 +-
 .../public/templates/index-page-template.html   |   2 +-
 .../templates/topology-page-template.html       |   6 +-
 storm-core/src/ui/public/topology.html          |  12 +-
 .../test/clj/backtype/storm/supervisor_test.clj |  12 +-
 .../test/clj/storm/trident/tuple_test.clj       |  28 +-
 storm-dist/binary/LICENSE                       |  10 +-
 storm-dist/binary/pom.xml                       |   2 +-
 storm-dist/binary/src/main/assembly/binary.xml  |  14 +
 storm-dist/source/pom.xml                       |   2 +-
 73 files changed, 3949 insertions(+), 717 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/README.markdown
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/bin/storm
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 1fe5b2e,cce89bc..c580bc8
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@@ -605,9 -501,8 +605,9 @@@
  (defmethod launch-worker
      :distributed [supervisor storm-id port worker-id]
      (let [conf (:conf supervisor)
 +          run-worker-as-user (conf SUPERVISOR-RUN-WORKER-AS-USER)
            storm-home (System/getProperty "storm.home")
-           storm-log-dir (or (System/getProperty "storm.log.dir") (str storm-home "/logs"))
+           storm-log-dir (or (System/getProperty "storm.log.dir") (str storm-home file-path-separator "logs"))
            stormroot (supervisor-stormdist-root conf storm-id)
            jlp (jlp stormroot conf)
            stormjar (supervisor-stormjar-path stormroot)
@@@ -638,7 -529,7 +638,7 @@@
                       (str "-Dlogfile.name=" logfilename)
                       (str "-Dstorm.home=" storm-home)
                       (str "-Dstorm.log.dir=" storm-log-dir)
-                      (str "-Dlogback.configurationFile=" storm-home "/logback/worker.xml")
 -                     (str "-Dlogback.configurationFile=" storm-home file-path-separator "logback" file-path-separator "cluster.xml")
++                     (str "-Dlogback.configurationFile=" storm-home file-path-separator "logback" file-path-separator "worker.xml")
                       (str "-Dstorm.id=" storm-id)
                       (str "-Dworker.id=" worker-id)
                       (str "-Dworker.port=" port)

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/backtype/storm/ui/core.clj
index 600c96a,969b514..e980a55
--- a/storm-core/src/clj/backtype/storm/ui/core.clj
+++ b/storm-core/src/clj/backtype/storm/ui/core.clj
@@@ -541,8 -519,9 +542,10 @@@
    ([summs]
     {"topologies"
      (for [^TopologySummary t summs]
-       {"id" (.get_id t)
+       {
+        "id" (.get_id t)
+        "encodedId" (url-encode (.get_id t))
 +       "owner" (.get_owner t)
         "name" (.get_name t)
         "status" (.get_status t)
         "uptime" (pretty-uptime-sec (.get_uptime_secs t))
@@@ -573,8 -551,9 +576,9 @@@
                        stats-seq include-sys?))
                last-error (most-recent-error (get errors id))
                error-host (get-error-host last-error)
 -              error-port (get-error-port last-error error-host top-id) ]]
 +              error-port (get-error-port last-error error-host top-id)]]
      {"spoutId" id
+      "encodedSpoutId" (url-encode id)
       "executors" (count summs)
       "tasks" (sum-tasks summs)
       "emitted" (get-in stats [:emitted window])
@@@ -596,8 -575,9 +600,9 @@@
                        stats-seq include-sys?))
                last-error (most-recent-error (get errors id))
                error-host (get-error-host last-error)
 -              error-port (get-error-port last-error error-host top-id) ]]
 +              error-port (get-error-port last-error error-host top-id)]]
      {"boltId" id
+      "encodedBoltId" (url-encode id)
       "executors" (count summs)
       "tasks" (sum-tasks summs)
       "emitted" (get-in stats [:emitted window])
@@@ -619,7 -599,7 +624,8 @@@
          workers (set (for [^ExecutorSummary e executors]
                         [(.get_host e) (.get_port e)]))]
        {"id" (.get_id summ)
+        "encodedId" (url-encode (.get_id summ))
 +       "owner" (.get_owner summ)
         "name" (.get_name summ)
         "status" (.get_status summ)
         "uptime" (pretty-uptime-sec (.get_uptime_secs summ))
@@@ -832,8 -811,8 +841,9 @@@
                       (= type :bolt) (bolt-stats window summ component summs include-sys?))
            errors (component-errors (get (.get_errors summ) component) topology-id)]
        (merge
 -       { "id" component
 +        {"user" user
 +         "id" component
+          "encodedId" (url-encode component)
           "name" (.get_name summ)
           "executors" (count summs)
           "tasks" (sum-tasks summs)
@@@ -869,67 -843,47 +880,59 @@@
    (GET "/api/v1/cluster/configuration" [& m]
         (json-response (cluster-configuration)
                        (:callback m) :serialize-fn identity))
 -  (GET "/api/v1/cluster/summary" [& m]
 -       (json-response (cluster-summary) (:callback m)))
 -  (GET "/api/v1/supervisor/summary" [& m]
 +  (GET "/api/v1/cluster/summary" [:as {:keys [cookies servlet-request]} & m]
 +       (let [user (.getUserName http-creds-handler servlet-request)]
 +         (assert-authorized-user servlet-request "getClusterInfo")
 +         (json-response (cluster-summary user) (:callback m))))
 +  (GET "/api/v1/supervisor/summary" [:as {:keys [cookies servlet-request]} & m]
 +       (assert-authorized-user servlet-request "getClusterInfo")
         (json-response (supervisor-summary) (:callback m)))
 -  (GET "/api/v1/topology/summary" [& m]
 +  (GET "/api/v1/topology/summary" [:as {:keys [cookies servlet-request]} & m]
 +       (assert-authorized-user servlet-request "getClusterInfo")
         (json-response (all-topologies-summary) (:callback m)))
 -  (GET  "/api/v1/topology/:id" [id & m]
 -          (json-response (topology-page id (:window m) (check-include-sys? (:sys m))) (:callback m)))
 +  (GET  "/api/v1/topology/:id" [:as {:keys [cookies servlet-request]} id & m]
-         (let [id (url-decode id)
-               user (.getUserName http-creds-handler servlet-request)]
++        (let [user (.getUserName http-creds-handler servlet-request)]
 +          (assert-authorized-user servlet-request "getTopology" (topology-config id))
 +          (json-response (topology-page id (:window m) (check-include-sys? (:sys m)) user) (:callback m))))
    (GET "/api/v1/topology/:id/visualization" [:as {:keys [cookies servlet-request]} id & m]
-         (let [id (url-decode id)]
-           (assert-authorized-user servlet-request "getTopology" (topology-config id))
-           (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m))) (:callback m))))
 -       (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m))) (:callback m)))
 -  (GET "/api/v1/topology/:id/component/:component" [id component & m]
 -        (json-response (component-page id component (:window m) (check-include-sys? (:sys m))) (:callback m)))
 -  (POST "/api/v1/topology/:id/activate" [id]
++        (assert-authorized-user servlet-request "getTopology" (topology-config id))
++        (json-response (mk-visualization-data id (:window m) (check-include-sys? (:sys m))) (:callback m)))
 +  (GET "/api/v1/topology/:id/component/:component" [:as {:keys [cookies servlet-request]} id component & m]
-        (let [id (url-decode id)
-              component (url-decode component)
-              user (.getUserName http-creds-handler servlet-request)]
++       (let [user (.getUserName http-creds-handler servlet-request)]
 +         (assert-authorized-user servlet-request "getTopology" (topology-config id))
 +         (json-response (component-page id component (:window m) (check-include-sys? (:sys m)) user) (:callback m))))
 +  (POST "/api/v1/topology/:id/activate" [:as {:keys [cookies servlet-request]} id]
      (with-nimbus nimbus
-       (let [id (url-decode id)
-             tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
              name (.get_name tplg)]
 +        (assert-authorized-user servlet-request "activate" (topology-config id))
          (.activate nimbus name)
          (log-message "Activating topology '" name "'")))
 -    (resp/redirect (str "/api/v1/topology/" (url-encode id))))
 -
 -  (POST "/api/v1/topology/:id/deactivate" [id]
 +    (resp/redirect (str "/api/v1/topology/" id)))
 +  (POST "/api/v1/topology/:id/deactivate" [:as {:keys [cookies servlet-request]} id]
      (with-nimbus nimbus
-       (let [id (url-decode id)
-             tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
              name (.get_name tplg)]
 +        (assert-authorized-user servlet-request "deactivate" (topology-config id))
          (.deactivate nimbus name)
          (log-message "Deactivating topology '" name "'")))
-     (resp/redirect (str "/api/v1/topology/" id)))
+     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
 -  (POST "/api/v1/topology/:id/rebalance/:wait-time" [id wait-time]
 +  (POST "/api/v1/topology/:id/rebalance/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
      (with-nimbus nimbus
-       (let [id (url-decode id)
-             tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
              name (.get_name tplg)
              options (RebalanceOptions.)]
 +        (assert-authorized-user servlet-request "rebalance" (topology-config id))
          (.set_wait_secs options (Integer/parseInt wait-time))
          (.rebalance nimbus name options)
          (log-message "Rebalancing topology '" name "' with wait time: " wait-time " secs")))
-     (resp/redirect (str "/api/v1/topology/" id)))
+     (resp/redirect (str "/api/v1/topology/" (url-encode id))))
 -  (POST "/api/v1/topology/:id/kill/:wait-time" [id wait-time]
 +  (POST "/api/v1/topology/:id/kill/:wait-time" [:as {:keys [cookies servlet-request]} id wait-time]
      (with-nimbus nimbus
-       (let [id (url-decode id)
-             tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
+       (let [tplg (.getTopologyInfo ^Nimbus$Client nimbus id)
              name (.get_name tplg)
              options (KillOptions.)]
 +        (assert-authorized-user servlet-request "killTopology" (topology-config id))
          (.set_wait_secs options (Integer/parseInt wait-time))
          (.killTopologyWithOpts nimbus name options)
          (log-message "Killing topology '" name "' with wait time: " wait-time " secs")))

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/src/ui/public/component.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/src/ui/public/js/script.js
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/templates/index-page-template.html
index 800b579,e3aab5e..9c48e83
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@@ -128,9 -118,8 +128,9 @@@
      <tbody>
        {{#topologies}}
        <tr>
-         <td><a href="/topology.html?id={{id}}">{{name}}</a></td>
+         <td><a href="/topology.html?id={{encodedId}}">{{name}}</a></td>
          <td>{{id}}</td>
 +        <td>{{owner}}</td>
          <td>{{status}}</td>
          <td>{{uptime}}</td>
          <td>{{workersTotal}}</td>

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-core/test/clj/backtype/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/backtype/storm/supervisor_test.clj
index 4261b4d,ba74d88..ebbf060
--- a/storm-core/test/clj/backtype/storm/supervisor_test.clj
+++ b/storm-core/test/clj/backtype/storm/supervisor_test.clj
@@@ -259,10 -253,10 +259,10 @@@
                                 opts
                                 topo-opts
                                 ["-Djava.library.path="
 -                                (str "-Dlogfile.name=worker-" mock-port ".log")
 +                                (str "-Dlogfile.name=" mock-storm-id "-worker-" mock-port ".log")
                                  "-Dstorm.home="
-                                 "-Dstorm.log.dir=/logs"
-                                 "-Dlogback.configurationFile=/logback/worker.xml"
+                                 (str "-Dstorm.log.dir=" file-path-separator "logs")
 -                                (str "-Dlogback.configurationFile=" file-path-separator "logback" file-path-separator "cluster.xml")
++                                (str "-Dlogback.configurationFile=" file-path-separator "logback" file-path-separator "worker.xml")
                                  (str "-Dstorm.id=" mock-storm-id)
                                  (str "-Dworker.id=" mock-worker-id)
                                  (str "-Dworker.port=" mock-port)
@@@ -322,10 -312,8 +322,10 @@@
            (stubbing [read-supervisor-storm-conf {TOPOLOGY-CLASSPATH topo-cp}
                       supervisor-stormdist-root nil
                       supervisor/jlp nil
 +                     set-worker-user! nil
 +                     supervisor/write-log-metadata! nil
                       launch-process nil
-                      current-classpath "/base"]
+                      current-classpath (str file-path-separator "base")]
                      (supervisor/launch-worker mock-supervisor
                                                mock-storm-id
                                                mock-port
@@@ -342,9 -329,7 +342,9 @@@
                       supervisor-stormdist-root nil
                       supervisor/jlp nil
                       launch-process nil
 +                     set-worker-user! nil
 +                     supervisor/write-log-metadata! nil
-                      current-classpath "/base"]
+                      current-classpath (str file-path-separator "base")]
                      (supervisor/launch-worker mock-supervisor
                                                mock-storm-id
                                                mock-port

http://git-wip-us.apache.org/repos/asf/storm/blob/ea0b8a2a/storm-dist/binary/src/main/assembly/binary.xml
----------------------------------------------------------------------


[30/50] git commit: STORM-519 adding tuple as an input param to HBaseValueMapper so user can chose to emit fields from original tuple in addition to lookup result.

Posted by bo...@apache.org.
STORM-519 adding tuple as an input param to HBaseValueMapper so user can chose to emit fields from original tuple in addition to lookup result.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/f87bb313
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/f87bb313
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/f87bb313

Branch: refs/heads/security
Commit: f87bb3135b4dcfd2b73689a35bf0b73a6cc5bc75
Parents: 1babd83
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Mon Oct 6 15:21:28 2014 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Mon Oct 6 15:21:28 2014 -0700

----------------------------------------------------------------------
 external/storm-hbase/pom.xml                                   | 2 +-
 .../main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java | 2 +-
 .../org/apache/storm/hbase/bolt/mapper/HBaseValueMapper.java   | 4 +++-
 .../java/org/apache/storm/hbase/trident/state/HBaseState.java  | 6 ++++--
 .../org/apache/storm/hbase/topology/WordCountValueMapper.java  | 3 ++-
 5 files changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/f87bb313/external/storm-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-hbase/pom.xml b/external/storm-hbase/pom.xml
index f976cea..a492164 100644
--- a/external/storm-hbase/pom.xml
+++ b/external/storm-hbase/pom.xml
@@ -21,7 +21,7 @@
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
-        <version>0.9.3-incubating-SNAPSHOT</version>
+        <version>0.9.3-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/storm/blob/f87bb313/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
----------------------------------------------------------------------
diff --git a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
index 12263a6..c6838be 100644
--- a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
+++ b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/HBaseLookupBolt.java
@@ -66,7 +66,7 @@ public class HBaseLookupBolt extends AbstractHBaseBolt {
 
         try {
             Result result = hBaseClient.batchGet(Lists.newArrayList(get))[0];
-            for(Values values : rowToTupleMapper.toValues(result)) {
+            for(Values values : rowToTupleMapper.toValues(tuple, result)) {
                 this.collector.emit(values);
             }
             this.collector.ack(tuple);

http://git-wip-us.apache.org/repos/asf/storm/blob/f87bb313/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/mapper/HBaseValueMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/mapper/HBaseValueMapper.java b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/mapper/HBaseValueMapper.java
index 39ce47a..bc38b83 100644
--- a/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/mapper/HBaseValueMapper.java
+++ b/external/storm-hbase/src/main/java/org/apache/storm/hbase/bolt/mapper/HBaseValueMapper.java
@@ -18,6 +18,7 @@
 package org.apache.storm.hbase.bolt.mapper;
 
 import backtype.storm.topology.OutputFieldsDeclarer;
+import backtype.storm.tuple.ITuple;
 import backtype.storm.tuple.Values;
 import org.apache.hadoop.hbase.client.Result;
 
@@ -27,11 +28,12 @@ import java.util.List;
 public interface HBaseValueMapper extends Serializable {
     /**
      *
+     * @param input tuple.
      * @param result HBase lookup result instance.
      * @return list of values that should be emitted by the lookup bolt.
      * @throws Exception
      */
-    public List<Values> toValues(Result result) throws Exception;
+    public List<Values> toValues(ITuple input, Result result) throws Exception;
 
     /**
      * declares the output fields for the lookup bolt.

http://git-wip-us.apache.org/repos/asf/storm/blob/f87bb313/external/storm-hbase/src/main/java/org/apache/storm/hbase/trident/state/HBaseState.java
----------------------------------------------------------------------
diff --git a/external/storm-hbase/src/main/java/org/apache/storm/hbase/trident/state/HBaseState.java b/external/storm-hbase/src/main/java/org/apache/storm/hbase/trident/state/HBaseState.java
index 66decf2..7b31fad 100644
--- a/external/storm-hbase/src/main/java/org/apache/storm/hbase/trident/state/HBaseState.java
+++ b/external/storm-hbase/src/main/java/org/apache/storm/hbase/trident/state/HBaseState.java
@@ -149,8 +149,10 @@ public class HBaseState implements State {
 
         try {
             Result[] results = hBaseClient.batchGet(gets);
-            for(Result result : results) {
-                List<Values> values = options.rowToStormValueMapper.toValues(result);
+            for(int i = 0; i < results.length; i++) {
+                Result result = results[i];
+                TridentTuple tuple = tridentTuples.get(i);
+                List<Values> values = options.rowToStormValueMapper.toValues(tuple, result);
                 batchRetrieveResult.add(values);
             }
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/storm/blob/f87bb313/external/storm-hbase/src/test/java/org/apache/storm/hbase/topology/WordCountValueMapper.java
----------------------------------------------------------------------
diff --git a/external/storm-hbase/src/test/java/org/apache/storm/hbase/topology/WordCountValueMapper.java b/external/storm-hbase/src/test/java/org/apache/storm/hbase/topology/WordCountValueMapper.java
index dd2ae20..2463085 100644
--- a/external/storm-hbase/src/test/java/org/apache/storm/hbase/topology/WordCountValueMapper.java
+++ b/external/storm-hbase/src/test/java/org/apache/storm/hbase/topology/WordCountValueMapper.java
@@ -20,6 +20,7 @@ package org.apache.storm.hbase.topology;
 
 import backtype.storm.topology.OutputFieldsDeclarer;
 import backtype.storm.tuple.Fields;
+import backtype.storm.tuple.ITuple;
 import backtype.storm.tuple.Values;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -51,7 +52,7 @@ import java.util.List;
 public class WordCountValueMapper implements HBaseValueMapper {
 
     @Override
-    public List<Values> toValues(Result result) throws Exception {
+    public List<Values> toValues(ITuple tuple, Result result) throws Exception {
         List<Values> values = new ArrayList<Values>();
         Cell[] cells = result.rawCells();
         for(Cell cell : cells) {


[04/50] git commit: Added STORM-493 to Changelog

Posted by bo...@apache.org.
Added STORM-493 to Changelog


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/46f05885
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/46f05885
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/46f05885

Branch: refs/heads/security
Commit: 46f058854d6ad4e5cad41739c397b703366eaf36
Parents: 7f29b73
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Sep 24 15:47:29 2014 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Sep 24 15:53:13 2014 -0500

----------------------------------------------------------------------
 CHANGELOG.md    | 1 +
 README.markdown | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/46f05885/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 24d89c3..10b70f1 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -75,6 +75,7 @@
  * STORM-386: nodejs multilang protocol implementation and examples
  * STORM-500: Add Spinner when UI is loading stats from nimbus
  * STORM-501: Missing StormSubmitter API
+ * STORM-493: Workers inherit storm.conf.file/storm.options properties of their supervisor
 
 ## 0.9.2-incubating
  * STORM-66: send taskid on initial handshake

http://git-wip-us.apache.org/repos/asf/storm/blob/46f05885/README.markdown
----------------------------------------------------------------------
diff --git a/README.markdown b/README.markdown
index 3deceb9..5fe638e 100644
--- a/README.markdown
+++ b/README.markdown
@@ -173,6 +173,7 @@ under the License.
 * Edison Xu ([@EdisonXu](https://github.com/EdisonXu))
 * Ariel Valentin ([@arielvalentin](http://blog.arielvalentin.com/))
 * Dane Hammer ([@danehammer](https://github.com/danehammer))
+* Christophe Carre' ([@chrisz](https://github.com/chrisz))
 
 ## Acknowledgements
 


[22/50] git commit: Merge branch 'storm-hbase' of github.com:ptgoetz/incubator-storm

Posted by bo...@apache.org.
Merge branch 'storm-hbase' of github.com:ptgoetz/incubator-storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9f712703
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9f712703
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9f712703

Branch: refs/heads/security
Commit: 9f712703e3c1017f79054a59230ce0a462504251
Parents: 0390891 4c515dd
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 15:59:44 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 15:59:44 2014 -0700

----------------------------------------------------------------------
 external/storm-hbase/LICENSE                    | 202 ++++++++++++++++
 external/storm-hbase/README.md                  | 213 +++++++++++++++++
 external/storm-hbase/pom.xml                    |  78 +++++++
 .../storm/hbase/bolt/AbstractHBaseBolt.java     |  69 ++++++
 .../org/apache/storm/hbase/bolt/HBaseBolt.java  |  77 ++++++
 .../storm/hbase/bolt/HBaseLookupBolt.java       |  83 +++++++
 .../storm/hbase/bolt/mapper/HBaseMapper.java    |  48 ++++
 .../bolt/mapper/HBaseProjectionCriteria.java    |  85 +++++++
 .../hbase/bolt/mapper/HBaseValueMapper.java     |  41 ++++
 .../hbase/bolt/mapper/SimpleHBaseMapper.java    |  90 +++++++
 .../apache/storm/hbase/common/ColumnList.java   | 202 ++++++++++++++++
 .../apache/storm/hbase/common/HBaseClient.java  | 135 +++++++++++
 .../org/apache/storm/hbase/common/IColumn.java  |  30 +++
 .../org/apache/storm/hbase/common/ICounter.java |  29 +++
 .../org/apache/storm/hbase/common/Utils.java    |  69 ++++++
 .../storm/hbase/security/HBaseSecurityUtil.java |  52 +++++
 .../mapper/SimpleTridentHBaseMapper.java        |  89 +++++++
 .../trident/mapper/TridentHBaseMapper.java      |  31 +++
 .../hbase/trident/state/HBaseMapState.java      | 233 +++++++++++++++++++
 .../storm/hbase/trident/state/HBaseQuery.java   |  40 ++++
 .../storm/hbase/trident/state/HBaseState.java   | 162 +++++++++++++
 .../hbase/trident/state/HBaseStateFactory.java  |  40 ++++
 .../storm/hbase/trident/state/HBaseUpdater.java |  32 +++
 .../storm/hbase/topology/LookupWordCount.java   |  79 +++++++
 .../hbase/topology/PersistentWordCount.java     |  81 +++++++
 .../storm/hbase/topology/TotalWordCounter.java  |  70 ++++++
 .../storm/hbase/topology/WordCountClient.java   |  57 +++++
 .../hbase/topology/WordCountValueMapper.java    |  69 ++++++
 .../storm/hbase/topology/WordCounter.java       |  59 +++++
 .../apache/storm/hbase/topology/WordSpout.java  |  88 +++++++
 .../storm/hbase/trident/PrintFunction.java      |  40 ++++
 .../storm/hbase/trident/WordCountTrident.java   | 104 +++++++++
 pom.xml                                         |   1 +
 storm-dist/binary/src/main/assembly/binary.xml  |  14 ++
 34 files changed, 2792 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9f712703/pom.xml
----------------------------------------------------------------------


[29/50] git commit: Added STORM-499 to changelog

Posted by bo...@apache.org.
Added STORM-499 to changelog


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/1babd831
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/1babd831
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/1babd831

Branch: refs/heads/security
Commit: 1babd831334d2654b50da7fc73c61d91905f768d
Parents: be66015
Author: Sriharsha Chintalapani <ha...@hortonworks.com>
Authored: Sat Oct 4 09:45:26 2014 -0700
Committer: Sriharsha Chintalapani <ha...@hortonworks.com>
Committed: Sat Oct 4 09:45:26 2014 -0700

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1babd831/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 9b28492..c2a9a70 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3-incubating
+ * STORM-499: Document and clean up shaded dependncy resolution with maven
  * STORM-210: Add storm-hbase module
  * STORM-507: Topology visualization should not block ui
  * STORM-504: Class used by `repl` command is deprecated.


[31/50] git commit: Merge branch 'STORM-439' of github.com:Parth-Brahmbhatt/incubator-storm

Posted by bo...@apache.org.
Merge branch 'STORM-439' of github.com:Parth-Brahmbhatt/incubator-storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/dbd431f3
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/dbd431f3
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/dbd431f3

Branch: refs/heads/security
Commit: dbd431f35e660631afee5f82b15f980e4216c7fd
Parents: 1babd83 55231b2
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 15:20:44 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 15:20:44 2014 -0400

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/ui/core.clj   |  41 +--
 storm-core/src/ui/public/component.html         |   8 +-
 storm-core/src/ui/public/js/purl.js             | 267 -------------------
 storm-core/src/ui/public/js/script.js           |   3 +-
 storm-core/src/ui/public/js/url.min.js          |   1 +
 storm-core/src/ui/public/js/visualization.js    |   2 +-
 .../templates/component-page-template.html      |   6 +-
 .../public/templates/index-page-template.html   |   2 +-
 .../templates/topology-page-template.html       |  14 +-
 storm-core/src/ui/public/topology.html          |   8 +-
 storm-dist/binary/LICENSE                       |  10 +-
 11 files changed, 49 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/dbd431f3/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/topology.html
index 477d3b0,6940df4..a6598ef
--- a/storm-core/src/ui/public/topology.html
+++ b/storm-core/src/ui/public/topology.html
@@@ -59,13 -59,11 +59,13 @@@
  <script>
  $(document).ajaxStop($.unblockUI);
  $(document).ajaxStart(function(){
 -    $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading topology summary...</h3>'});
 +    if ($("#topology-visualization").children().size() == 0) {
 +        $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading topology summary...</h3>'});
 +    }
  });
  $(document).ready(function() {
-     var topologyId = $.url().param("id");
-     var window = $.url().param("window");
+     var topologyId = $.url("?id");
+     var window = $.url("?window");
      var sys = $.cookies.get("sys") || "false";
      var url = "/api/v1/topology/"+topologyId+"?sys="+sys;
      if(window) url += "&window="+window;


[14/50] git commit: update scm and mailing lists to reflect graduation

Posted by bo...@apache.org.
update scm and mailing lists to reflect graduation


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/5d1c7c32
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/5d1c7c32
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/5d1c7c32

Branch: refs/heads/security
Commit: 5d1c7c327ac804e99e75a19d8c51e79371b8f2f6
Parents: c683d45
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Sep 30 11:43:25 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Sep 30 11:43:25 2014 -0700

----------------------------------------------------------------------
 pom.xml | 24 ++++++++++++------------
 1 file changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5d1c7c32/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0db0ea4..038a635 100644
--- a/pom.xml
+++ b/pom.xml
@@ -31,7 +31,7 @@
     <packaging>pom</packaging>
     <name>Storm</name>
     <description>Distributed and fault-tolerant realtime computation</description>
-    <url>http://storm.incubator.apache.org</url>
+    <url>http://storm.apache.org</url>
     <licenses>
         <license>
             <name>The Apache Software License, Version 2.0</name>
@@ -42,17 +42,17 @@
     <mailingLists>
         <mailingList>
             <name>Storm user mailing list</name>
-            <subscribe>user-subscribe@storm.incubator.apache.org</subscribe>
-            <unsubscribe>user-unsubscribe@storm.incubator.apache.org</unsubscribe>
-            <post>user@storm.incubator.apache.org</post>
-            <archive>http://mail-archives.apache.org/mod_mbox/incubator-storm-user/</archive>
+            <subscribe>user-subscribe@storm.apache.org</subscribe>
+            <unsubscribe>user-unsubscribe@storm.apache.org</unsubscribe>
+            <post>user@storm.apache.org</post>
+            <archive>http://mail-archives.apache.org/mod_mbox/storm-user/</archive>
         </mailingList>
         <mailingList>
             <name>Storm developer mailing list</name>
-            <subscribe>dev-subscribe@storm.incubator.apache.org</subscribe>
-            <unsubscribe>dev-unsubscribe@storm.incubator.apache.org</unsubscribe>
-            <post>dev@storm.incubator.apache.org</post>
-            <archive>http://mail-archives.apache.org/mod_mbox/incubator-storm-dev/</archive>
+            <subscribe>dev-subscribe@storm.apache.org</subscribe>
+            <unsubscribe>dev-unsubscribe@storm.apache.org</unsubscribe>
+            <post>dev@storm.apache.org</post>
+            <archive>http://mail-archives.apache.org/mod_mbox/storm-dev/</archive>
         </mailingList>
     </mailingLists>
 
@@ -163,10 +163,10 @@
     </modules>
 
     <scm>
-        <connection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-storm.git</connection>
-        <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-storm.git</developerConnection>
+        <connection>scm:git:https://git-wip-us.apache.org/repos/asf/storm.git</connection>
+        <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/storm.git</developerConnection>
         <tag>HEAD</tag>
-        <url>https://git-wip-us.apache.org/repos/asf/incubator-storm</url>
+        <url>https://git-wip-us.apache.org/repos/asf/storm</url>
     </scm>
 
     <issueManagement>


[02/50] git commit: Merge branch 'patch-1' of http://github.com/danehammer/incubator-storm into pull-247

Posted by bo...@apache.org.
Merge branch 'patch-1' of http://github.com/danehammer/incubator-storm into pull-247

Conflicts:
	README.markdown


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b0974a37
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b0974a37
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b0974a37

Branch: refs/heads/security
Commit: b0974a37ba4abeb834d1d1729ad11f6b4e43fa28
Parents: cfcedcf 766106a
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Sep 24 14:38:46 2014 -0500
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Sep 24 14:38:46 2014 -0500

----------------------------------------------------------------------
 README.markdown | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b0974a37/README.markdown
----------------------------------------------------------------------
diff --cc README.markdown
index fa8d539,ac99b48..3deceb9
--- a/README.markdown
+++ b/README.markdown
@@@ -170,8 -169,7 +170,9 @@@ under the License
  * Dan Blanchard ([@dan-blanchard](https://github.com/dan-blanchard))
  * Mykhailo "Michael" Pershyn ([@pershyn](https://github.com/pershyn))
  * Dave Golombek ([@dgolombek](https://github.com/dgolombek))
 +* Edison Xu ([@EdisonXu](https://github.com/EdisonXu))
 +* Ariel Valentin ([@arielvalentin](http://blog.arielvalentin.com/))
+ * Dane Hammer ([@danehammer](https://github.com/danehammer))
  
  ## Acknowledgements
  


[38/50] git commit: Merge branch 'STORM-506' of github.com:d2r/storm

Posted by bo...@apache.org.
Merge branch 'STORM-506' of github.com:d2r/storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9e63e85e
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9e63e85e
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9e63e85e

Branch: refs/heads/security
Commit: 9e63e85eaa538e79882bbe1ae749a533d368ffd3
Parents: 588fd37 985fb1f
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:33:05 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:33:05 2014 -0400

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/ui/core.clj | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9e63e85e/storm-core/src/clj/backtype/storm/ui/core.clj
----------------------------------------------------------------------


[35/50] git commit: Merge PR #253. This closes #253.

Posted by bo...@apache.org.
Merge PR #253. This closes #253.


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/8079245b
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/8079245b
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/8079245b

Branch: refs/heads/security
Commit: 8079245ba9688321da1c090e648caa77430cbff2
Parents: 84afd45
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:14:32 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:14:32 2014 -0400

----------------------------------------------------------------------
 storm-core/src/clj/backtype/storm/daemon/supervisor.clj | 10 +++-------
 storm-core/src/dev/resources/storm.py                   |  1 -
 storm-core/src/dev/resources/storm.rb                   |  1 -
 3 files changed, 3 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/8079245b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
index 8c18e1c..cce89bc 100644
--- a/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/backtype/storm/daemon/supervisor.clj
@@ -502,9 +502,7 @@
     :distributed [supervisor storm-id port worker-id]
     (let [conf (:conf supervisor)
           storm-home (System/getProperty "storm.home")
-          storm-options (System/getProperty "storm.options")
-          storm-conf-file (System/getProperty "storm.conf.file")
-          storm-log-dir (or (System/getProperty "storm.log.dir") (str storm-home "/logs"))
+          storm-log-dir (or (System/getProperty "storm.log.dir") (str storm-home file-path-separator "logs"))
           stormroot (supervisor-stormdist-root conf storm-id)
           jlp (jlp stormroot conf)
           stormjar (supervisor-stormjar-path stormroot)
@@ -530,10 +528,8 @@
                     [(str "-Djava.library.path=" jlp)
                      (str "-Dlogfile.name=" logfilename)
                      (str "-Dstorm.home=" storm-home)
-                     (str "-Dstorm.conf.file=" storm-conf-file)
-                     (str "-Dstorm.options=" storm-options)
                      (str "-Dstorm.log.dir=" storm-log-dir)
-                     (str "-Dlogback.configurationFile=" storm-home "/logback/cluster.xml")
+                     (str "-Dlogback.configurationFile=" storm-home file-path-separator "logback" file-path-separator "cluster.xml")
                      (str "-Dstorm.id=" storm-id)
                      (str "-Dworker.id=" worker-id)
                      (str "-Dworker.port=" port)
@@ -629,4 +625,4 @@
         ))))
 
 (defn -main []
-  (-launch (standalone-supervisor)))
+  (-launch (standalone-supervisor)))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/8079245b/storm-core/src/dev/resources/storm.py
----------------------------------------------------------------------
diff --git a/storm-core/src/dev/resources/storm.py b/storm-core/src/dev/resources/storm.py
deleted file mode 120000
index 5e73111..0000000
--- a/storm-core/src/dev/resources/storm.py
+++ /dev/null
@@ -1 +0,0 @@
-../../multilang/py/storm.py
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/8079245b/storm-core/src/dev/resources/storm.rb
----------------------------------------------------------------------
diff --git a/storm-core/src/dev/resources/storm.rb b/storm-core/src/dev/resources/storm.rb
deleted file mode 120000
index 96db018..0000000
--- a/storm-core/src/dev/resources/storm.rb
+++ /dev/null
@@ -1 +0,0 @@
-../../multilang/rb/storm.rb
\ No newline at end of file


[19/50] git commit: Merge branch 'STORM-428-fixed' of github.com:wurstmeister/incubator-storm

Posted by bo...@apache.org.
Merge branch 'STORM-428-fixed' of github.com:wurstmeister/incubator-storm

Conflicts:
	CHANGELOG.md


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3f2eec5f
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3f2eec5f
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3f2eec5f

Branch: refs/heads/security
Commit: 3f2eec5ff3db9cdb72bd34027c9441e22e599dc2
Parents: f72dad8 9553b14
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 15:48:40 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 15:48:40 2014 -0700

----------------------------------------------------------------------
 CHANGELOG.md                                    |   1 +
 .../src/test/storm/kafka/TridentKafkaTest.java  |   9 +-
 .../src/jvm/backtype/storm/tuple/ITuple.java    | 136 +++++++++++++++
 .../src/jvm/backtype/storm/tuple/Tuple.java     | 113 +------------
 .../storm/trident/testing/MockTridentTuple.java | 166 -------------------
 .../jvm/storm/trident/tuple/TridentTuple.java   |  46 +----
 .../storm/trident/tuple/TridentTupleView.java   |  35 +++-
 .../jvm/storm/trident/tuple/ValuePointer.java   |   1 -
 .../test/clj/storm/trident/tuple_test.clj       |  28 ++--
 9 files changed, 196 insertions(+), 339 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3f2eec5f/CHANGELOG.md
----------------------------------------------------------------------
diff --cc CHANGELOG.md
index 9cc13ef,605a7da..c6371a6
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@@ -78,7 -77,7 +78,8 @@@
   * STORM-501: Missing StormSubmitter API
   * STORM-493: Workers inherit storm.conf.file/storm.options properties of their supervisor
   * STORM-498: make ZK connection timeout configurable in Kafka spout
+  * STORM-428: extracted ITuple interface
 + * STORM-508: Update DEVELOPER.md now that Storm has graduated from Incubator
  
  ## 0.9.2-incubating
   * STORM-66: send taskid on initial handshake


[17/50] git commit: update changelog for STORM-504

Posted by bo...@apache.org.
update changelog for STORM-504


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/39ca6f7b
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/39ca6f7b
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/39ca6f7b

Branch: refs/heads/security
Commit: 39ca6f7bd0caf0bec8ba3be5d955860085ba2460
Parents: f3421fb
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 15:43:39 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 15:43:39 2014 -0700

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/39ca6f7b/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 24950b4..9cc13ef 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3-incubating
+ * STORM-504: Class used by `repl` command is deprecated.
  * STORM-330: Implement storm exponential backoff stategy for netty client and curator
  * STORM-461: exit-process! does not always exit the process, but throws an exception
  * STORM-341: fix assignment sorting


[43/50] git commit: remove storm.js symlink

Posted by bo...@apache.org.
remove storm.js symlink


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/e9953aff
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/e9953aff
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/e9953aff

Branch: refs/heads/security
Commit: e9953aff4a8c42ec363cd1d81719c970446dc5b6
Parents: 533f7ba
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:50:51 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:50:51 2014 -0400

----------------------------------------------------------------------
 storm-core/src/dev/resources/storm.js | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e9953aff/storm-core/src/dev/resources/storm.js
----------------------------------------------------------------------
diff --git a/storm-core/src/dev/resources/storm.js b/storm-core/src/dev/resources/storm.js
deleted file mode 120000
index a5fc98b..0000000
--- a/storm-core/src/dev/resources/storm.js
+++ /dev/null
@@ -1 +0,0 @@
-../../multilang/js/storm.js
\ No newline at end of file


[36/50] git commit: add non-symlink versions of storm.py and storm.rb

Posted by bo...@apache.org.
add non-symlink versions of storm.py and storm.rb


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/fc641a2d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/fc641a2d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/fc641a2d

Branch: refs/heads/security
Commit: fc641a2da3079eba7c2980ce7ff5afcee8be407e
Parents: 8079245
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:20:17 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:20:17 2014 -0400

----------------------------------------------------------------------
 storm-core/src/dev/resources/storm.py | 247 +++++++++++++++++++++++++++++
 storm-core/src/dev/resources/storm.rb | 227 ++++++++++++++++++++++++++
 2 files changed, 474 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/fc641a2d/storm-core/src/dev/resources/storm.py
----------------------------------------------------------------------
diff --git a/storm-core/src/dev/resources/storm.py b/storm-core/src/dev/resources/storm.py
new file mode 100755
index 0000000..d2a3082
--- /dev/null
+++ b/storm-core/src/dev/resources/storm.py
@@ -0,0 +1,247 @@
+# -*- coding: utf-8 -*-
+
+# 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.
+
+import sys
+import os
+import traceback
+from collections import deque
+
+try:
+    import simplejson as json
+except ImportError:
+    import json
+
+json_encode = lambda x: json.dumps(x)
+json_decode = lambda x: json.loads(x)
+
+#reads lines and reconstructs newlines appropriately
+def readMsg():
+    msg = ""
+    while True:
+        line = sys.stdin.readline()
+        if not line:
+            raise Exception('Read EOF from stdin')
+        if line[0:-1] == "end":
+            break
+        msg = msg + line
+    return json_decode(msg[0:-1])
+
+MODE = None
+ANCHOR_TUPLE = None
+
+#queue up commands we read while trying to read taskids
+pending_commands = deque()
+
+def readTaskIds():
+    if pending_taskids:
+        return pending_taskids.popleft()
+    else:
+        msg = readMsg()
+        while type(msg) is not list:
+            pending_commands.append(msg)
+            msg = readMsg()
+        return msg
+
+#queue up taskids we read while trying to read commands/tuples
+pending_taskids = deque()
+
+def readCommand():
+    if pending_commands:
+        return pending_commands.popleft()
+    else:
+        msg = readMsg()
+        while type(msg) is list:
+            pending_taskids.append(msg)
+            msg = readMsg()
+        return msg
+
+def readTuple():
+    cmd = readCommand()
+    return Tuple(cmd["id"], cmd["comp"], cmd["stream"], cmd["task"], cmd["tuple"])
+
+def sendMsgToParent(msg):
+    print json_encode(msg)
+    print "end"
+    sys.stdout.flush()
+
+def sync():
+    sendMsgToParent({'command':'sync'})
+
+def sendpid(heartbeatdir):
+    pid = os.getpid()
+    sendMsgToParent({'pid':pid})
+    open(heartbeatdir + "/" + str(pid), "w").close()
+
+def emit(*args, **kwargs):
+    __emit(*args, **kwargs)
+    return readTaskIds()
+
+def emitDirect(task, *args, **kwargs):
+    kwargs["directTask"] = task
+    __emit(*args, **kwargs)
+
+def __emit(*args, **kwargs):
+    global MODE
+    if MODE == Bolt:
+        emitBolt(*args, **kwargs)
+    elif MODE == Spout:
+        emitSpout(*args, **kwargs)
+
+def emitBolt(tup, stream=None, anchors = [], directTask=None):
+    global ANCHOR_TUPLE
+    if ANCHOR_TUPLE is not None:
+        anchors = [ANCHOR_TUPLE]
+    m = {"command": "emit"}
+    if stream is not None:
+        m["stream"] = stream
+    m["anchors"] = map(lambda a: a.id, anchors)
+    if directTask is not None:
+        m["task"] = directTask
+    m["tuple"] = tup
+    sendMsgToParent(m)
+
+def emitSpout(tup, stream=None, id=None, directTask=None):
+    m = {"command": "emit"}
+    if id is not None:
+        m["id"] = id
+    if stream is not None:
+        m["stream"] = stream
+    if directTask is not None:
+        m["task"] = directTask
+    m["tuple"] = tup
+    sendMsgToParent(m)
+
+def ack(tup):
+    sendMsgToParent({"command": "ack", "id": tup.id})
+
+def fail(tup):
+    sendMsgToParent({"command": "fail", "id": tup.id})
+
+def reportError(msg):
+    sendMsgToParent({"command": "error", "msg": msg})
+
+def log(msg, level=2):
+    sendMsgToParent({"command": "log", "msg": msg, "level":level})
+
+def logTrace(msg):
+    log(msg, 0)
+
+def logDebug(msg):
+    log(msg, 1)
+
+def logInfo(msg):
+    log(msg, 2)
+
+def logWarn(msg):
+    log(msg, 3)
+
+def logError(msg):
+    log(msg, 4)
+
+def rpcMetrics(name, params):
+    sendMsgToParent({"command": "metrics", "name": name, "params": params})
+
+def initComponent():
+    setupInfo = readMsg()
+    sendpid(setupInfo['pidDir'])
+    return [setupInfo['conf'], setupInfo['context']]
+
+class Tuple(object):
+    def __init__(self, id, component, stream, task, values):
+        self.id = id
+        self.component = component
+        self.stream = stream
+        self.task = task
+        self.values = values
+
+    def __repr__(self):
+        return '<%s%s>' % (
+                self.__class__.__name__,
+                ''.join(' %s=%r' % (k, self.__dict__[k]) for k in sorted(self.__dict__.keys())))
+
+class Bolt(object):
+    def initialize(self, stormconf, context):
+        pass
+
+    def process(self, tuple):
+        pass
+
+    def run(self):
+        global MODE
+        MODE = Bolt
+        conf, context = initComponent()
+        try:
+            self.initialize(conf, context)
+            while True:
+                tup = readTuple()
+                self.process(tup)
+        except Exception, e:
+            reportError(traceback.format_exc(e))
+
+class BasicBolt(object):
+    def initialize(self, stormconf, context):
+        pass
+
+    def process(self, tuple):
+        pass
+
+    def run(self):
+        global MODE
+        MODE = Bolt
+        global ANCHOR_TUPLE
+        conf, context = initComponent()
+        try:
+            self.initialize(conf, context)
+            while True:
+                tup = readTuple()
+                ANCHOR_TUPLE = tup
+                self.process(tup)
+                ack(tup)
+        except Exception, e:
+            reportError(traceback.format_exc(e))
+
+class Spout(object):
+    def initialize(self, conf, context):
+        pass
+
+    def ack(self, id):
+        pass
+
+    def fail(self, id):
+        pass
+
+    def nextTuple(self):
+        pass
+
+    def run(self):
+        global MODE
+        MODE = Spout
+        conf, context = initComponent()
+        try:
+            self.initialize(conf, context)
+            while True:
+                msg = readCommand()
+                if msg["command"] == "next":
+                    self.nextTuple()
+                if msg["command"] == "ack":
+                    self.ack(msg["id"])
+                if msg["command"] == "fail":
+                    self.fail(msg["id"])
+                sync()
+        except Exception, e:
+            reportError(traceback.format_exc(e))

http://git-wip-us.apache.org/repos/asf/storm/blob/fc641a2d/storm-core/src/dev/resources/storm.rb
----------------------------------------------------------------------
diff --git a/storm-core/src/dev/resources/storm.rb b/storm-core/src/dev/resources/storm.rb
new file mode 100644
index 0000000..17232d1
--- /dev/null
+++ b/storm-core/src/dev/resources/storm.rb
@@ -0,0 +1,227 @@
+# -*- coding: utf-8 -*-
+
+# 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.
+
+require "rubygems"
+require "json"
+
+module Storm
+  module Protocol
+    class << self
+      attr_accessor :mode, :pending_taskids, :pending_commands
+    end
+
+    self.pending_taskids = []
+    self.pending_commands = []
+
+    def read_message
+      msg = ""
+      loop do
+        line = STDIN.readline.chomp
+        break if line == "end"
+        msg << line
+        msg << "\n"
+      end
+      JSON.parse msg.chomp
+    end
+
+    def read_task_ids
+      Storm::Protocol.pending_taskids.shift ||
+        begin
+          msg = read_message
+          until msg.is_a? Array
+            Storm::Protocol.pending_commands.push(msg)
+            msg = read_message
+          end
+          msg
+        end
+    end
+
+    def read_command
+      Storm::Protocol.pending_commands.shift ||
+        begin
+          msg = read_message
+          while msg.is_a? Array
+            Storm::Protocol.pending_taskids.push(msg)
+            msg = read_message
+          end
+          msg
+        end
+    end
+
+    def send_msg_to_parent(msg)
+      puts msg.to_json
+      puts "end"
+      STDOUT.flush
+    end
+
+    def sync
+      send_msg_to_parent({'command' => 'sync'})
+    end
+
+    def send_pid(heartbeat_dir)
+      pid = Process.pid
+      send_msg_to_parent({'pid' => pid})
+      File.open("#{heartbeat_dir}/#{pid}", "w").close
+    end
+
+    def emit_bolt(tup, args = {})
+      stream = args[:stream]
+      anchors = args[:anchors] || args[:anchor] || []
+      anchors = [anchors] unless anchors.is_a? Enumerable
+      direct = args[:direct_task]
+      m = {:command => :emit, :anchors => anchors.map(&:id), :tuple => tup}
+      m[:stream] = stream if stream
+      m[:task] = direct if direct
+      send_msg_to_parent m
+      read_task_ids unless direct
+    end
+
+    def emit_spout(tup, args = {})
+      stream = args[:stream]
+      id = args[:id]
+      direct = args[:direct_task]
+      m = {:command => :emit, :tuple => tup}
+      m[:id] = id if id
+      m[:stream] = stream if stream
+      m[:task] = direct if direct
+      send_msg_to_parent m
+      read_task_ids unless direct
+    end
+
+    def emit(*args)
+      case Storm::Protocol.mode
+      when 'spout'
+        emit_spout(*args)
+      when 'bolt'
+        emit_bolt(*args)
+      end
+    end
+
+    def ack(tup)
+      send_msg_to_parent :command => :ack, :id => tup.id
+    end
+
+    def fail(tup)
+      send_msg_to_parent :command => :fail, :id => tup.id
+    end
+
+    def reportError(msg)
+      send_msg_to_parent :command => :error, :msg => msg.to_s
+    end
+
+    def log(msg, level=2)
+      send_msg_to_parent :command => :log, :msg => msg.to_s, :level => level
+    end
+
+    def logTrace(msg)
+      log(msg, 0)
+    end
+
+    def logDebug(msg)
+      log(msg, 1)
+    end
+
+    def logInfo(msg)
+      log(msg, 2)
+    end
+
+    def logWarn(msg)
+      log(msg, 3)
+    end
+
+    def logError(msg)
+      log(msg, 4)
+    end
+
+    def handshake
+      setup_info = read_message
+      send_pid setup_info['pidDir']
+      [setup_info['conf'], setup_info['context']]
+    end
+  end
+
+  class Tuple
+    attr_accessor :id, :component, :stream, :task, :values
+
+    def initialize(id, component, stream, task, values)
+      @id = id
+      @component = component
+      @stream = stream
+      @task = task
+      @values = values
+    end
+
+    def self.from_hash(hash)
+      Tuple.new(*hash.values_at("id", "comp", "stream", "task", "tuple"))
+    end
+  end
+
+  class Bolt
+    include Storm::Protocol
+
+    def prepare(conf, context); end
+
+    def process(tuple); end
+
+    def run
+      Storm::Protocol.mode = 'bolt'
+      prepare(*handshake)
+      begin
+        while true
+          process Tuple.from_hash(read_command)
+        end
+      rescue Exception => e
+        reportError 'Exception in bolt: ' + e.message + ' - ' + e.backtrace.join('\n')
+      end
+    end
+  end
+
+  class Spout
+    include Storm::Protocol
+
+    def open(conf, context); end
+
+    def nextTuple; end
+
+    def ack(id); end
+
+    def fail(id); end
+
+    def run
+      Storm::Protocol.mode = 'spout'
+      open(*handshake)
+
+      begin
+        while true
+          msg = read_command
+          case msg['command']
+          when 'next'
+            nextTuple
+          when 'ack'
+            ack(msg['id'])
+          when 'fail'
+            fail(msg['id'])
+          end
+          sync
+        end
+      rescue Exception => e
+        reportError 'Exception in spout: ' + e.message + ' - ' + e.backtrace.join('\n')
+      end
+    end
+  end
+end


[21/50] git commit: update changelog for STORM-507

Posted by bo...@apache.org.
update changelog for STORM-507


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/0390891b
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/0390891b
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/0390891b

Branch: refs/heads/security
Commit: 0390891b5178919bc0b92ca27a0a25b314a7e3a6
Parents: a49aebb
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Oct 1 15:58:01 2014 -0700
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Oct 1 15:58:01 2014 -0700

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0390891b/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index c6371a6..dc7ff8e 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3-incubating
+ * STORM-507: Topology visualization should not block ui
  * STORM-504: Class used by `repl` command is deprecated.
  * STORM-330: Implement storm exponential backoff stategy for netty client and curator
  * STORM-461: exit-process! does not always exit the process, but throws an exception


[37/50] git commit: update changelog for STORM-490

Posted by bo...@apache.org.
update changelog for STORM-490


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/588fd378
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/588fd378
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/588fd378

Branch: refs/heads/security
Commit: 588fd378d33bbd4de4a77e602a74706e61e489e9
Parents: fc641a2
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:32:58 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:32:58 2014 -0400

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/588fd378/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 9830e89..d019262 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 0.9.3
+ * STORM-490: fix build under Windows
  * STORM-439: Replace purl.js qith jquery URL plugin
  * STORM-499: Document and clean up shaded dependncy resolution with maven
  * STORM-210: Add storm-hbase module


[40/50] git commit: Merge branch 'STORM-488' of github.com:miguno/storm

Posted by bo...@apache.org.
Merge branch 'STORM-488' of github.com:miguno/storm


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9b754ee4
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9b754ee4
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9b754ee4

Branch: refs/heads/security
Commit: 9b754ee4309786fc4c58468679506d57cd8a5072
Parents: d50b772 b3528c7
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 16:42:59 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 16:42:59 2014 -0400

----------------------------------------------------------------------
 bin/storm | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[03/50] git commit: Merge remote-tracking branch 'upstream/master' into STORM-439

Posted by bo...@apache.org.
Merge remote-tracking branch 'upstream/master' into STORM-439

Conflicts:
	storm-core/src/ui/public/js/visualization.js
	storm-core/src/ui/public/templates/component-page-template.html
	storm-core/src/ui/public/templates/index-page-template.html
	storm-core/src/ui/public/templates/topology-page-template.html


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/55231b27
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/55231b27
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/55231b27

Branch: refs/heads/security
Commit: 55231b27922a7cccf7592ebc82de37791a097a07
Parents: cccd4a5 b0974a3
Author: Parth Brahmbhatt <br...@gmail.com>
Authored: Wed Sep 24 13:43:09 2014 -0700
Committer: Parth Brahmbhatt <br...@gmail.com>
Committed: Wed Sep 24 13:43:09 2014 -0700

----------------------------------------------------------------------
 CHANGELOG.md                                    |  17 +
 DEVELOPER.md                                    |   2 +-
 LICENSE                                         |  25 +
 README.markdown                                 |   7 +-
 STORM-UI-REST-API.md                            |   4 +-
 dev-tools/jira-github-join.py                   |   2 +-
 .../multilang/resources/asyncSplitsentence.js   |  32 +
 .../multilang/resources/randomsentence.js       |  75 +++
 .../multilang/resources/splitsentence.js        |  26 +
 .../storm-starter/multilang/resources/storm.js  | 349 +++++++++++
 .../storm/starter/WordCountTopologyNode.java    | 121 ++++
 .../src/jvm/storm/kafka/KafkaUtils.java         |   5 +-
 .../src/jvm/storm/kafka/PartitionManager.java   |   5 +-
 pom.xml                                         |   2 +-
 storm-core/pom.xml                              |  53 +-
 .../src/clj/backtype/storm/daemon/common.clj    |   3 +
 .../src/clj/backtype/storm/daemon/nimbus.clj    |   4 +-
 storm-core/src/clj/backtype/storm/event.clj     |   3 +
 .../backtype/storm/scheduler/EvenScheduler.clj  |   2 +-
 storm-core/src/clj/backtype/storm/testing.clj   |   2 +-
 storm-core/src/clj/backtype/storm/util.clj      |   2 +-
 storm-core/src/dev/resources/storm.js           | 349 +++++++++++
 storm-core/src/dev/resources/tester_bolt.js     |  27 +
 storm-core/src/dev/resources/tester_spout.js    |  58 ++
 .../src/jvm/backtype/storm/StormSubmitter.java  |  16 +
 .../backtype/storm/messaging/netty/Client.java  |  27 +-
 .../StormBoundedExponentialBackoffRetry.java    |  74 +++
 .../src/jvm/backtype/storm/utils/Utils.java     |  30 +-
 storm-core/src/ui/public/component.html         |   6 +
 storm-core/src/ui/public/images/spinner.gif     | Bin 0 -> 23470 bytes
 storm-core/src/ui/public/index.html             |   6 +
 .../src/ui/public/js/jquery.blockUI.min.js      |   6 +
 storm-core/src/ui/public/js/script.js           |  11 +
 storm-core/src/ui/public/js/visualization.js    |  14 +-
 .../templates/component-page-template.html      | 593 ++++++++++++++-----
 .../public/templates/index-page-template.html   | 219 +++++--
 .../templates/topology-page-template.html       | 195 +++++-
 storm-core/src/ui/public/topology.html          |   6 +
 .../test/clj/backtype/storm/multilang_test.clj  |  55 +-
 .../test/clj/backtype/storm/scheduler_test.clj  |  21 +
 .../test/clj/backtype/storm/utils_test.clj      |   3 +-
 ...StormBoundedExponentialBackoffRetryTest.java | 101 ++++
 42 files changed, 2265 insertions(+), 293 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/55231b27/storm-core/src/ui/public/component.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/component.html
index 21a5e76,6fa998f..fddb9ef
--- a/storm-core/src/ui/public/component.html
+++ b/storm-core/src/ui/public/component.html
@@@ -23,8 -24,9 +24,9 @@@
  <script src="/js/jquery.tablesorter.min.js" type="text/javascript"></script>
  <script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
  <script src="/js/jquery.mustache.js" type="text/javascript"></script>
 -<script src="/js/purl.js" type="text/javascript"></script>
 +<script src="/js/url.min.js" type="text/javascript"></script>
  <script src="/js/bootstrap-twipsy.js" type="text/javascript"></script>
+ <script src="/js/jquery.blockUI.min.js" type="text/javascript"></script>
  <script src="/js/script.js" type="text/javascript"></script>
  <script src="/js/moment.min.js" type="text/javascript"></script>
  </head>
@@@ -46,10 -48,14 +48,14 @@@
  </div>
  <p id="toggle-switch" style="display: block;" class="js-only"></p>
  <script>
+ $(document).ajaxStop($.unblockUI);
+ $(document).ajaxStart(function(){
+     $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading component summary...</h3>'});
+ });
  $(document).ready(function() {
 -    var componentId = $.url().param("id");
 -    var topologyId = $.url().param("topology_id");
 -    var window = $.url().param("window");
 +    var componentId = $.url("?id");
 +    var topologyId = $.url("?topology_id");
 +    var window = $.url("?window");
      var sys = $.cookies.get("sys") || "false";
      var url = "/api/v1/topology/"+topologyId+"/component/"+componentId+"?sys="+sys;
      if(window) url += "&window="+window;

http://git-wip-us.apache.org/repos/asf/storm/blob/55231b27/storm-core/src/ui/public/js/script.js
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/55231b27/storm-core/src/ui/public/js/visualization.js
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/js/visualization.js
index 6d00901,7cec697..1e1c4d6
--- a/storm-core/src/ui/public/js/visualization.js
+++ b/storm-core/src/ui/public/js/visualization.js
@@@ -389,10 -389,11 +389,11 @@@ function show_visualization(sys) 
  
      should_update = true;
      var update_freq_ms = 10000;
-     var update = function(should_rechoose){
+     var update = function(should_rechoose) {
+       if(should_update) {
          $.ajax({
 -            url: "/api/v1/topology/" + $.url().param("id") + "/visualization",
 +            url: "/api/v1/topology/"+$.url("?id")+"/visualization",
-             success: function(data, status, jqXHR) {
+             success: function (data, status, jqXHR) {
                  topology_data = data;
                  update_data(topology_data, sys);
                  sys.renderer.signal_update();

http://git-wip-us.apache.org/repos/asf/storm/blob/55231b27/storm-core/src/ui/public/templates/component-page-template.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/templates/component-page-template.html
index f912bd3,f718967..1e916e3
--- a/storm-core/src/ui/public/templates/component-page-template.html
+++ b/storm-core/src/ui/public/templates/component-page-template.html
@@@ -15,159 -15,474 +15,474 @@@
   limitations under the License.
  -->
  <script id="component-summary-template" type="text/html">
- <h2>Component summary</h2>
- <table><thead><tr><th><span class="tip right" title="The ID assigned to a the Component by the Topology.">Id</span></th><th><span class="tip above" title="The name given to the topology by when it was submitted. Click the name to view the Topology's information.">Topology</span></th><th><span class="tip above" title="Executors are threads in a Worker process.">Executors</span></th><th><span class="tip above" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">Tasks</span></th></tr></thead>
- <tbody>
- <tr>
- <td>{{id}}</td>
- <td><a href="/topology.html?id={{encodedTopologyId}}">{{name}}</a></td>
- <td>{{executors}}</td>
- <td>{{tasks}}</td>
- </tbody>
- </table>
+   <h2>Component summary</h2>
+   <table>
+     <thead>
+       <tr>
+         <th>
+           <span class="tip right" title="The ID assigned to a the Component by the Topology.">
+             Id
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="The name given to the topology by when it was submitted. Click the name to view the Topology's information.">
+             Topology
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="Executors are threads in a Worker process.">
+             Executors
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">
+             Tasks
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       <tr>
+         <td>{{id}}</td>
 -        <td><a href="/topology.html?id={{topologyId}}">{{name}}</a></td>
++        <td><a href="/topology.html?id={{encodedTopologyId}}">{{name}}</a></td>
+         <td>{{executors}}</td>
+         <td>{{tasks}}</td>
+     </tbody>
+   </table>
  </script>
  <script id="spout-stats-detail-template" type="text/html">
- <h2>Spout stats</h2>
- <table class="zebra-striped" id="spout-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The past period of time for which the statistics apply. Click on a value to set the window for this page.">Window</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span data-original-title="The number of Tuples emitted that sent to one or more bolts." class="tip above">Transferred</span></th><th class="header"><span class="tip above" title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">Complete latency (ms)</span></th><th class="header"><span class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">Acked</span></th><th class="header"><span data-original-title="The number of Tuple &quot;trees&quot; that were 
 explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">Failed</span></th></tr></thead>
- <tbody>
- {{#spoutSummary}}
- <tr>
- <td><a href="/component.html?id={{encodedId}}&topology_id={{encodedTopologyId}}&window={{window}}">{{windowPretty}}</a></td>
- <td>{{transferred}}</td>
- <td>{{emitted}}</td>
- <td>{{completeLatency}}</td>
- <td>{{acked}}</td>
- <td>{{failed}}</td>
- </tr>
- {{/spoutSummary}}
- </tbody>
- </table>
+   <h2>Spout stats</h2>
+   <table class="zebra-striped" id="spout-stats-table">
+     <thead>
+       <tr>
+         <th class="header headerSortDown">
+           <span class="tip right" title="The past period of time for which the statistics apply. Click on a value to set the window for this page.">
+             Window
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted.">
+             Emitted
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuples emitted that sent to one or more bolts." class="tip above">
+             Transferred
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">
+             Complete latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">
+             Acked
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuple &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">
+             Failed
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#spoutSummary}}
+       <tr>
 -        <td><a href="/component.html?id={{id}}&topology_id={{topologyId}}&window={{window}}">{{windowPretty}}</a></td>
++        <td><a href="/component.html?id={{encodedId}}&topology_id={{encodedTopologyId}}&window={{window}}">{{windowPretty}}</a></td>
+         <td>{{transferred}}</td>
+         <td>{{emitted}}</td>
+         <td>{{completeLatency}}</td>
+         <td>{{acked}}</td>
+         <td>{{failed}}</td>
+       </tr>
+       {{/spoutSummary}}
+     </tbody>
+   </table>
  </script>
  <script id="output-stats-template" type="text/html">
- <h2>Output stats ({{windowHint}})</h2>
- <table class="zebra-striped" id="output-stats-table"><thead><tr><th class="header headerSortDown"><span data-original-title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given." class="tip right">Stream</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span data-original-title="The number of Tuples emitted that sent to one or more bolts." class="tip above">Transferred</span></th><th class="header"><span data-original-title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done." class="tip above">Complete latency (ms)</span></th><th class="header"><span data-original-title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done." class="tip above">Acked</span></th><th class="header"><span data-original-title="The number of Tupl
 e &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">Failed</span></th></tr></thead>
- <tbody>
- {{#outputStats}}
- <tr>
- <td>{{stream}}</td>
- <td>{{emitted}}</td>
- <td>{{transferred}}</td>
- <td>{{completeLatency}}</td>
- <td>{{acked}}</td>
- <td>{{failed}}</td>
- </tr>
- {{/outputStats}}
- </tbody>
- </table>
+   <h2>Output stats ({{windowHint}})</h2>
+   <table class="zebra-striped" id="output-stats-table">
+     <thead>
+       <tr>
+         <th class="header headerSortDown">
+           <span data-original-title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given." class="tip right">
+             Stream
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted.">
+             Emitted
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuples emitted that sent to one or more bolts." class="tip above">
+             Transferred
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done." class="tip above">
+             Complete latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done." class="tip above">
+             Acked
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuple &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">
+             Failed
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#outputStats}}
+       <tr>
+         <td>{{stream}}</td>
+         <td>{{emitted}}</td>
+         <td>{{transferred}}</td>
+         <td>{{completeLatency}}</td>
+         <td>{{acked}}</td>
+         <td>{{failed}}</td>
+       </tr>
+       {{/outputStats}}
+     </tbody>
+   </table>
  </script>
  <script id="executor-stats-template" type="text/html">
- <h2>Executors ({{windowHint}})</h2>
- <table class="zebra-striped" id="executor-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The unique executor ID.">Id</span></th><th class="header"><span class="tip right" title="The length of time an Executor (thread) has been alive.">Uptime</span></th><th class="header"><span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">Host</span></th><th class="header"><span data-original-title="The port number used by the Worker to which an Executor is assigned. Click on the port number to open the logviewer page for this Worker." class="tip above">Port</span></th><th class="header"><span data-original-title="The number of Tuples emitted." class="tip above">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span class="tip abov
 e" title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">Complete latency (ms)</span></th><th class="header"><span class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">Acked</span></th><th class="header"><span data-original-title="The number of Tuple &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">Failed</span></th></tr></thead>
- <tbody>
- {{#executorStats}}
- <tr>
- <td>{{id}}</td>
- <td>{{uptime}}</td>
- <td>{{host}}</td>
- <td><a href="{{workerLogLink}}">{{port}}</a></td>
- <td>{{emitted}}</td>
- <td>{{transferred}}</td>
- <td>{{completeLatency}}</td>
- <td>{{acked}}</td>
- <td>{{failed}}</td>
- </tr>
- {{/executorStats}}
- </tbody>
- </table>
+   <h2>Executors ({{windowHint}})</h2>
+   <table class="zebra-striped" id="executor-stats-table">
+     <thead>
+       <tr>
+         <th class="header headerSortDown">
+           <span class="tip right" title="The unique executor ID.">
+             Id
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip right" title="The length of time an Executor (thread) has been alive.">
+             Uptime
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">
+             Host
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The port number used by the Worker to which an Executor is assigned. Click on the port number to open the logviewer page for this Worker." class="tip above">
+             Port
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuples emitted." class="tip above">
+             Emitted
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">
+             Transferred
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The average time a Tuple &quot;tree&quot; takes to be completely processed by the Topology. A value of 0 is expected if no acking is done.">
+             Complete latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuple &quot;trees&quot; successfully processed. A value of 0 is expected if no acking is done.">
+             Acked
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuple &quot;trees&quot; that were explicitly failed or timed out before acking was completed. A value of 0 is expected if no acking is done." class="tip left">
+             Failed
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#executorStats}}
+       <tr>
+         <td>{{id}}</td>
+         <td>{{uptime}}</td>
+         <td>{{host}}</td>
+         <td><a href="{{workerLogLink}}">{{port}}</a></td>
+         <td>{{emitted}}</td>
+         <td>{{transferred}}</td>
+         <td>{{completeLatency}}</td>
+         <td>{{acked}}</td>
+         <td>{{failed}}</td>
+       </tr>
+       {{/executorStats}}
+     </tbody>
+   </table>
  </script>
  <script id="bolt-stats-template" type="text/html">
- <h2>Bolt stats</h2>
- <table class="zebra-striped" id="bolt-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The past period of time for which the statistics apply. Click on a value to set the window for this page.">Window</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span data-original-title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple." class="tip above">Execute latency (ms)</span></th><th class="header"><span class="tip above" title="The number of incoming Tuples processed.">Executed</span></th><th class="header"><span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple unt
 il a number of other Tuples have been received." class="tip above">Process latency (ms)</span></th><th class="header"><span data-original-title="The number of Tuples acknowledged by this Bolt." class="tip above">Acked</span></th><th class="header"><span data-original-title="The number of tuples Failed by this Bolt." class="tip left">Failed</span></th></tr></thead>
- <tbody>
- {{#boltStats}}
- <tr>
- <td><a href="/component.html?id={{encodedId}}&topology_id={{encodedTopologyId}}&window={{window}}">{{windowPretty}}</a></td>
- <td>{{emitted}}</td>
- <td>{{transferred}}</td>
- <td>{{executeLatency}}</td>
- <td>{{executed}}</td>
- <td>{{processLatency}}</td>
- <td>{{acked}}</td>
- <td>{{failed}}</td>
- </tr>
- {{/boltStats}}
- </tbody>
- </table>
+   <h2>Bolt stats</h2>
+   <table class="zebra-striped" id="bolt-stats-table">
+     <thead>
+       <tr>
+         <th class="header headerSortDown">
+           <span class="tip right" title="The past period of time for which the statistics apply. Click on a value to set the window for this page.">
+             Window
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted.">
+             Emitted
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">
+             Transferred
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple." class="tip above">
+             Execute latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of incoming Tuples processed.">
+             Executed
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received." class="tip above">
+             Process latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuples acknowledged by this Bolt." class="tip above">
+             Acked
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of tuples Failed by this Bolt." class="tip left">
+             Failed
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#boltStats}}
+       <tr>
 -        <td><a href="/component.html?id={{id}}&topology_id={{topologyId}}&window={{window}}">{{windowPretty}}</a></td>
++        <td><a href="/component.html?id={{encodedId}}&topology_id={{encodedTopologyId}}&window={{window}}">{{windowPretty}}</a></td>
+         <td>{{emitted}}</td>
+         <td>{{transferred}}</td>
+         <td>{{executeLatency}}</td>
+         <td>{{executed}}</td>
+         <td>{{processLatency}}</td>
+         <td>{{acked}}</td>
+         <td>{{failed}}</td>
+       </tr>
+       {{/boltStats}}
+     </tbody>
+   </table>
  </script>
  <script id="bolt-input-stats-template" type="text/html">
- <h2>Input stats ({{windowHint}})</h2>
- <table class="zebra-striped" id="bolt-input-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The ID assigned to a the Component by the Topology.">Component</span></th><th class="header"><span class="tip right" title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given.">Stream</span></th><th class="header"><span class="tip above" title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple.">Execute latency (ms)</span></th><th class="header"><span class="tip above" title="The number of incoming Tuples processed.">Executed</span></th><th class="header"><span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received." class="tip above">Process latency (ms)</span></th><th class="header"><span class="t
 ip above" title="The number of Tuples acknowledged by this Bolt.">Acked</span></th><th class="header"><span data-original-title="The number of tuples Failed by this Bolt." class="tip left">Failed</span></th></tr></thead>
- <tbody>
- {{#inputStats}}
- <tr>
- <td>{{component}}</td>
- <td>{{stream}}</td>
- <td>{{executeLatency}}</td>
- <td>{{executed}}</td>
- <td>{{processLatency}}</td>
- <td>{{acked}}</td>
- <td>{{failed}}</td>
- </tr>
- {{/inputStats}}
- </tbody>
- </table>
+   <h2>Input stats ({{windowHint}})</h2>
+   <table class="zebra-striped" id="bolt-input-stats-table">
+     <thead>
+       <tr>
+         <th class="header headerSortDown">
+           <span class="tip right" title="The ID assigned to a the Component by the Topology.">
+             Component
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip right" title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given.">
+             Stream
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple.">
+             Execute latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of incoming Tuples processed.">
+             Executed
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received." class="tip above">
+             Process latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples acknowledged by this Bolt.">
+             Acked
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of tuples Failed by this Bolt." class="tip left">
+             Failed
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#inputStats}}
+       <tr>
+         <td>{{component}}</td>
+         <td>{{stream}}</td>
+         <td>{{executeLatency}}</td>
+         <td>{{executed}}</td>
+         <td>{{processLatency}}</td>
+         <td>{{acked}}</td>
+         <td>{{failed}}</td>
+       </tr>
+       {{/inputStats}}
+     </tbody>
+   </table>
  </script>
  <script id="bolt-output-stats-template" type="text/html">
- <h2>Output stats ({{windowHint}})</h2>
- <table class="zebra-striped" id="bolt-output-stats-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given.">Stream</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th></tr></thead>
- <tbody>
- {{#outputStats}}
- <tr>
- <td>{{stream}}</td>
- <td>{{emitted}}</td>
- <td>{{transferred}}</td>
- </tr>
- {{/outputStats}}
- </tbody>
- </table>
+   <h2>Output stats ({{windowHint}})</h2>
+   <table class="zebra-striped" id="bolt-output-stats-table">
+     <thead>
+       <tr>
+         <th class="header headerSortDown">
+           <span class="tip right" title="The name of the Tuple stream given in the Topolgy, or &quot;default&quot; if none was given.">
+             Stream
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted.">
+             Emitted
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">
+             Transferred
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#outputStats}}
+       <tr>
+         <td>{{stream}}</td>
+         <td>{{emitted}}</td>
+         <td>{{transferred}}</td>
+       </tr>
+       {{/outputStats}}
+     </tbody>
+   </table>
  </script>
  <script id="bolt-executor-template" type="text/html">
- <h2>Executors</h2>
- <table class="zebra-striped" id="bolt-executor-table"><thead><tr><th class="header headerSortDown"><span class="tip right" title="The unique executor ID.">Id</span></th><th class="header"><span data-original-title="The length of time an Executor (thread) has been alive." class="tip right">Uptime</span></th><th class="header"><span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">Host</span></th><th class="header"><span class="tip above" title="The port number used by the Worker to which an Executor is assigned. Click on the port number to open the logviewer page for this Worker.">Port</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted.">Emitted</span></th><th class="header"><span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">Transferred</span></th><th class="header"><span class="tip above" title="If th
 is is around 1.0, the corresponding Bolt is running as fast as it can, so you may want to increase the Bolt's parallelism. This is (number executed * average execute latency) / measurement time.">Capacity (last 10m)</span></th><th class="header"><span data-original-title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple." class="tip above">Execute latency (ms)</span></th><th class="header"><span class="tip above" title="The number of incoming Tuples processed.">Executed</span></th><th class="header"><span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received." class="tip above">Process latency (ms)</span></th><th class="header"><span data-original-title="The number of Tuples acknowledged by this Bolt." class="tip above">Acked</span></th><th class="header"><span data-or
 iginal-title="The number of tuples Failed by this Bolt." class="tip left">Failed</span></th></tr></thead>
- <tbody>
- {{#executorStats}}
- <tr>
- <td>{{id}}}</td>
- <td>{{uptime}}</td>
- <td>{{host}}</td>
- <td><a href="{{workerLogLink}}">{{port}}</a></td>
- <td>{{emitted}}</td>
- <td>{{transferred}}</td>
- <td>{{capacity}}</td>
- <td>{{executeLatency}}</td>
- <td>{{executed}}</td>
- <td>{{processLatency}}</td>
- <td>{{acked}}</td>
- <td>{{failed}}</td>
- </tr>
- {{/executorStats}}
- </tbody>
- </table>
+   <h2>Executors</h2>
+   <table class="zebra-striped" id="bolt-executor-table">
+     <thead>
+       <tr>
+         <th class="header headerSortDown">
+           <span class="tip right" title="The unique executor ID.">
+             Id
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The length of time an Executor (thread) has been alive." class="tip right">
+             Uptime
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">
+             Host
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The port number used by the Worker to which an Executor is assigned. Click on the port number to open the logviewer page for this Worker.">
+             Port
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted.">
+             Emitted
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of Tuples emitted that sent to one or more bolts.">
+             Transferred
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="If this is around 1.0, the corresponding Bolt is running as fast as it can, so you may want to increase the Bolt's parallelism. This is (number executed * average execute latency) / measurement time.">
+             Capacity (last 10m)
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The average time a Tuple spends in the execute method. The execute method may complete without sending an Ack for the tuple." class="tip above">
+             Execute latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span class="tip above" title="The number of incoming Tuples processed.">
+             Executed
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The average time it takes to Ack a Tuple after it is first received.  Bolts that join, aggregate or batch may not Ack a tuple until a number of other Tuples have been received." class="tip above">
+             Process latency (ms)
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of Tuples acknowledged by this Bolt." class="tip above">
+             Acked
+           </span>
+         </th>
+         <th class="header">
+           <span data-original-title="The number of tuples Failed by this Bolt." class="tip left">
+             Failed
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#executorStats}}
+       <tr>
+         <td>{{id}}}</td>
+         <td>{{uptime}}</td>
+         <td>{{host}}</td>
+         <td><a href="{{workerLogLink}}">{{port}}</a></td>
+         <td>{{emitted}}</td>
+         <td>{{transferred}}</td>
+         <td>{{capacity}}</td>
+         <td>{{executeLatency}}</td>
+         <td>{{executed}}</td>
+         <td>{{processLatency}}</td>
+         <td>{{acked}}</td>
+         <td>{{failed}}</td>
+       </tr>
+       {{/executorStats}}
+     </tbody>
+   </table>
  </script>
  <script id="component-errors-template" type="text/html">
- <h2>Errors</h2>
- <table class="zebra-striped" id="component-errors-table"><thead><tr><th>Time</th><th>Error Host</th><th>Error Port</th><th>Error</th></tr></thead>
- <tbody>
- {{#componentErrors}}
- <tr>
- <td>{{time}}</td>
- <td>{{errorHost}}</td>
- <td><a href="{{errorWorkerLogLink}}">{{errorPort}}</a></td>
- <td>
-   <span id="{{errorLapsedSecs}}" class="errorSpan">{{error}}</span>
- </td>
- </tr>
- {{/componentErrors}}
- </tbody>
- </table>
+   <h2>Errors</h2>
+   <table class="zebra-striped" id="component-errors-table">
+     <thead>
+       <tr>
+         <th>Time</th>
+         <th>Error Host</th>
+         <th>Error Port</th>
+         <th>Error</th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#componentErrors}}
+       <tr>
+         <td>{{time}}</td>
+         <td>{{errorHost}}</td>
+         <td><a href="{{errorWorkerLogLink}}">{{errorPort}}</a></td>
+         <td>
+           <span id="{{errorLapsedSecs}}" class="errorSpan">{{error}}</span>
+         </td>
+       </tr>
+       {{/componentErrors}}
+     </tbody>
+   </table>
  </script>

http://git-wip-us.apache.org/repos/asf/storm/blob/55231b27/storm-core/src/ui/public/templates/index-page-template.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/templates/index-page-template.html
index d0db1bf,d249fac..e3aab5e
--- a/storm-core/src/ui/public/templates/index-page-template.html
+++ b/storm-core/src/ui/public/templates/index-page-template.html
@@@ -31,36 -75,103 +75,103 @@@
  </table>
  </script>
  <script id="topology-summary-template" type="text/html">
- <table class="zebra-striped" id="topology-summary-table">
- <thead><tr><th><span class="tip right" title="The name given to the topology by when it was submitted. Click the name to view the Topology's information.">Name</span></th><th><span class="tip right" title="The unique ID given to a Topology each time it is launched.">Id</span></th><th><span class="tip above" title="The status can be one of ACTIVE, INACTIVE, KILLED, or REBALANCING.">Status</span></th><th><span class="tip above" title="The time since the Topology was submitted.">Uptime</span></th><th><span class="tip above" title="The number of Workers (processes).">Num workers</span></th><th><span class="tip above" title="Executors are threads in a Worker process.">Num executors</span></th><th><span class="tip above" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">Num tasks</span></th></tr></thead>
- <tbody>
- {{#topologies}}
- <tr>
-   <td><a href="/topology.html?id={{encodedId}}">{{name}}</a></td>
-   <td>{{id}}</td>
-   <td>{{status}}</td>
-   <td>{{uptime}}</td>
-   <td>{{workersTotal}}</td>
-   <td>{{executorsTotal}}</td>
-   <td>{{tasksTotal}}</td>
- </tr>
- {{/topologies}}
- </tbody>
- </table>
+   <table class="zebra-striped" id="topology-summary-table">
+     <thead>
+       <tr>
+         <th>
+           <span class="tip right" title="The name given to the topology by when it was submitted. Click the name to view the Topology's information.">
+             Name
+           </span>
+         </th>
+         <th>
+           <span class="tip right" title="The unique ID given to a Topology each time it is launched.">
+             Id
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="The status can be one of ACTIVE, INACTIVE, KILLED, or REBALANCING.">
+             Status
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="The time since the Topology was submitted.">
+             Uptime
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="The number of Workers (processes).">
+             Num workers
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="Executors are threads in a Worker process.">
+             Num executors
+           </span>
+         </th>
+         <th>
+           <span class="tip above" title="A Task is an instance of a Bolt or Spout. The number of Tasks is almost always equal to the number of Executors.">
+             Num tasks
+           </span>
+         </th>
+       </tr>
+     </thead>
+     <tbody>
+       {{#topologies}}
+       <tr>
 -        <td><a href="/topology.html?id={{id}}">{{name}}</a></td>
++        <td><a href="/topology.html?id={{encodedId}}">{{name}}</a></td>
+         <td>{{id}}</td>
+         <td>{{status}}</td>
+         <td>{{uptime}}</td>
+         <td>{{workersTotal}}</td>
+         <td>{{executorsTotal}}</td>
+         <td>{{tasksTotal}}</td>
+       </tr>
+       {{/topologies}}
+     </tbody>
+   </table>
  </script>
  <script id="supervisor-summary-template" type="text/html">
- <table class="zebra-striped" id="supervisor-summary-table"><thead><tr><th><span class="tip right" title="A unique identifier given to a Supervisor when it joins the cluster.">Id</span></th><th><span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">Host</span></th><th><span class="tip above" title="The length of time a Supervisor has been registered to the cluster.">Uptime</span></th><th><span class="tip above" title="Slots are Workers (processes).">Slots</span></th><th><span class="tip left" title="Slots are Workers (processes).">Used slots</span></th></tr></thead>
- <tbody>
- {{#supervisors}}
- <tr>
-   <td>{{id}}</td>
-   <td>{{host}}</td>
-   <td>{{uptime}}</td>
-   <td>{{slotsTotal}}</td>
-   <td>{{slotsUsed}}</td>
- </tr>
- {{/supervisors}}
- </tbody>
+ <table class="zebra-striped" id="supervisor-summary-table">
+   <thead>
+     <tr>
+       <th>
+         <span class="tip right" title="A unique identifier given to a Supervisor when it joins the cluster.">
+           Id
+         </span>
+       </th>
+       <th>
+         <span class="tip above" title="The hostname reported by the remote host. (Note that this hostname is not the result of a reverse lookup at the Nimbus node.)">
+           Host
+         </span>
+       </th>
+       <th>
+         <span class="tip above" title="The length of time a Supervisor has been registered to the cluster.">
+           Uptime
+         </span>
+       </th>
+       <th>
+         <span class="tip above" title="Slots are Workers (processes).">
+           Slots
+         </span>
+       </th>
+       <th>
+         <span class="tip left" title="Slots are Workers (processes).">
+           Used slots
+         </span>
+       </th>
+     </tr>
+   </thead>
+   <tbody>
+     {{#supervisors}}
+     <tr>
+       <td>{{id}}</td>
+       <td>{{host}}</td>
+       <td>{{uptime}}</td>
+       <td>{{slotsTotal}}</td>
+       <td>{{slotsUsed}}</td>
+     </tr>
+     {{/supervisors}}
+   </tbody>
  </table>
  </script>
  

http://git-wip-us.apache.org/repos/asf/storm/blob/55231b27/storm-core/src/ui/public/templates/topology-page-template.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/templates/topology-page-template.html
index c5b3db0,b977071..3479fc1
--- a/storm-core/src/ui/public/templates/topology-page-template.html
+++ b/storm-core/src/ui/public/templates/topology-page-template.html
@@@ -148,5 -326,8 +326,8 @@@
  </script>
  
  <script id="topology-actions-template" type="text/html">
-   <input {{activateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'activate', false, 0)" type="button" value="Activate"><input {{deactivateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'deactivate', false, 0)" type="button" value="Deactivate"><input {{rebalanceStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'rebalance', true, {{msgTimeout}})" type="button" value="Rebalance"><input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30)" type="button" value="Kill">
 -  <input {{activateStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'activate', false, 0)" type="button" value="Activate">
 -  <input {{deactivateStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'deactivate', false, 0)" type="button" value="Deactivate">
 -  <input {{rebalanceStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'rebalance', true, {{msgTimeout}})" type="button" value="Rebalance">
 -  <input {{killStatus}} onclick="confirmAction('{{id}}', '{{name}}', 'kill', true, 30)" type="button" value="Kill">
++  <input {{activateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'activate', false, 0)" type="button" value="Activate">
++  <input {{deactivateStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'deactivate', false, 0)" type="button" value="Deactivate">
++  <input {{rebalanceStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'rebalance', true, {{msgTimeout}})" type="button" value="Rebalance">
++  <input {{killStatus}} onclick="confirmAction('{{encodedId}}', '{{name}}', 'kill', true, 30)" type="button" value="Kill">
  </script>

http://git-wip-us.apache.org/repos/asf/storm/blob/55231b27/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------
diff --cc storm-core/src/ui/public/topology.html
index e4da3c5,9d53144..6940df4
--- a/storm-core/src/ui/public/topology.html
+++ b/storm-core/src/ui/public/topology.html
@@@ -24,8 -25,9 +25,9 @@@
  <script src="/js/jquery.tablesorter.min.js" type="text/javascript"></script>
  <script src="/js/jquery.cookies.2.2.0.min.js" type="text/javascript"></script>
  <script src="/js/jquery.mustache.js" type="text/javascript"></script>
 -<script src="/js/purl.js" type="text/javascript"></script>
 +<script src="/js/url.min.js" type="text/javascript"></script>
  <script src="/js/bootstrap-twipsy.js" type="text/javascript"></script>
+ <script src="/js/jquery.blockUI.min.js" type="text/javascript"></script>
  <script src="/js/script.js" type="text/javascript"></script>
  <script src="/js/visualization.js" type="text/javascript"></script>
  <script src="/js/arbor.js" type="text/javascript"></script>
@@@ -55,9 -57,13 +57,13 @@@
  </div>
  </body>
  <script>
+ $(document).ajaxStop($.unblockUI);
+ $(document).ajaxStart(function(){
+     $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading topology summary...</h3>'});
+ });
  $(document).ready(function() {
 -    var topologyId = $.url().param("id");
 -    var window = $.url().param("window");
 +    var topologyId = $.url("?id");
 +    var window = $.url("?window");
      var sys = $.cookies.get("sys") || "false";
      var url = "/api/v1/topology/"+topologyId+"?sys="+sys;
      if(window) url += "&window="+window;


[08/50] git commit: Topology visualization should not block ui

Posted by bo...@apache.org.
Topology visualization should not block ui


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3c18fd76
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3c18fd76
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3c18fd76

Branch: refs/heads/security
Commit: 3c18fd76971f00a1c691d1c1b74f320115c4bc72
Parents: cfcedcf
Author: Kishor Patil <kp...@yahoo-inc.com>
Authored: Thu Sep 25 13:33:38 2014 -0500
Committer: Kishor Patil <kp...@yahoo-inc.com>
Committed: Thu Sep 25 13:33:38 2014 -0500

----------------------------------------------------------------------
 storm-core/src/ui/public/topology.html | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3c18fd76/storm-core/src/ui/public/topology.html
----------------------------------------------------------------------
diff --git a/storm-core/src/ui/public/topology.html b/storm-core/src/ui/public/topology.html
index 9d53144..477d3b0 100644
--- a/storm-core/src/ui/public/topology.html
+++ b/storm-core/src/ui/public/topology.html
@@ -59,7 +59,9 @@
 <script>
 $(document).ajaxStop($.unblockUI);
 $(document).ajaxStart(function(){
-    $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading topology summary...</h3>'});
+    if ($("#topology-visualization").children().size() == 0) {
+        $.blockUI({ message: '<img src="images/spinner.gif" /> <h3>Loading topology summary...</h3>'});
+    }
 });
 $(document).ready(function() {
     var topologyId = $.url().param("id");


[32/50] git commit: update LICENSE to reflect removal of purl.js

Posted by bo...@apache.org.
update LICENSE to reflect removal of purl.js


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/53245c25
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/53245c25
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/53245c25

Branch: refs/heads/security
Commit: 53245c25ffb7a41de76ebf5c0558286a04a47d35
Parents: dbd431f
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 15:24:09 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 15:24:09 2014 -0400

----------------------------------------------------------------------
 LICENSE | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/53245c25/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index d15be6c..e30786b 100644
--- a/LICENSE
+++ b/LICENSE
@@ -371,16 +371,6 @@ For Arbor Tween (storm-core/src/ui/public/js/arbor-tween.js)
 
 -----------------------------------------------------------------------
 
-For Purl (storm-core/src/ui/public/js/purl.js)
-
-Purl (A JavaScript URL parser) v2.3.1
-Developed and maintanined by Mark Perkins, mark@allmarkedup.com
-Source repository: https://github.com/allmarkedup/jQuery-URL-Parser
-Licensed under an MIT-style license. See https://github.com/allmarkedup/jQuery-URL-Parser/blob/master/LICENSE for details.
-
-
------------------------------------------------------------------------
-
 For mustache.js (storm-core/src/ui/public/js/jquery.mustache.js)
 
 The MIT License
@@ -433,8 +423,18 @@ WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING
 FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
 OTHER DEALINGS IN THE SOFTWARE.
 
+
 -----------------------------------------------------------------------
 
+For Jquery url plugin (storm-core/src/ui/public/js/url.min.js)
+
+Jquery Url (A Jquery plugin for URL parser) v1.8.6
+Source repository: https://github.com/websanova/js-url
+Licensed under an MIT-style license. Seehttps://github.com/websanova/js-url#license for details.
+
+-----------------------------------------------------------------------
+
+
 For jquery.blockUI.min.js (storm-core/src/ui/public/js/jquery.blockUI.min.js)
 
 jQuery BlockUI; v20131009


[34/50] git commit: remove 'incubating' from 0.9.3 release

Posted by bo...@apache.org.
remove 'incubating' from 0.9.3 release


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/84afd458
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/84afd458
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/84afd458

Branch: refs/heads/security
Commit: 84afd4582bdf538975c40bd5f36c6b4f02c605c4
Parents: 88b85e6
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Oct 7 15:26:35 2014 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Oct 7 15:26:35 2014 -0400

----------------------------------------------------------------------
 CHANGELOG.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/84afd458/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 8e46c64..9830e89 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,4 @@
-## 0.9.3-incubating
+## 0.9.3
  * STORM-439: Replace purl.js qith jquery URL plugin
  * STORM-499: Document and clean up shaded dependncy resolution with maven
  * STORM-210: Add storm-hbase module