You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by pt...@apache.org on 2018/01/22 17:41:07 UTC

[01/38] storm git commit: STORM-2153: New Metrics Reporting API

Repository: storm
Updated Branches:
  refs/heads/1.x-branch 25fa9dd7c -> c4404cab6


STORM-2153: New Metrics Reporting API


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

Branch: refs/heads/1.x-branch
Commit: fa1e59f4408f4017d4b6c69e672eb7c27d68f3a7
Parents: e85b64a
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Jul 11 13:58:16 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Jul 11 13:58:16 2017 -0400

----------------------------------------------------------------------
 conf/defaults.yaml                              |  25 ++++
 .../storm/starter/ExclamationTopology.java      |   2 +-
 .../apache/storm/starter/ReliableWordCount.java | 121 +++++++++++++++++
 .../apache/storm/starter/WordCountTopology.java |   2 +-
 external/storm-autocreds/pom.xml                |   4 +-
 .../hdfs/avro/ConfluentAvroSerializer.java      |   2 +-
 pom.xml                                         |  10 ++
 storm-core/pom.xml                              |  17 ++-
 .../clj/org/apache/storm/daemon/executor.clj    |  18 ++-
 .../src/clj/org/apache/storm/daemon/task.clj    |   7 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  12 +-
 .../src/clj/org/apache/storm/disruptor.clj      |   5 +-
 storm-core/src/jvm/org/apache/storm/Config.java |   3 +
 .../apache/storm/metrics2/DisruptorMetrics.java |  93 +++++++++++++
 .../org/apache/storm/metrics2/SimpleGauge.java  |  38 ++++++
 .../storm/metrics2/StormMetricRegistry.java     | 133 +++++++++++++++++++
 .../reporters/ConsoleStormReporter.java         |  63 +++++++++
 .../metrics2/reporters/CsvStormReporter.java    |  93 +++++++++++++
 .../reporters/GangliaStormReporter.java         | 133 +++++++++++++++++++
 .../reporters/GraphiteStormReporter.java        | 100 ++++++++++++++
 .../metrics2/reporters/JmxStormReporter.java    |  88 ++++++++++++
 .../reporters/SheduledStormReporter.java        |  71 ++++++++++
 .../storm/metrics2/reporters/StormReporter.java |  32 +++++
 .../org/apache/storm/task/TopologyContext.java  |  26 ++++
 .../org/apache/storm/utils/DisruptorQueue.java  |  43 ++++--
 .../utils/DisruptorQueueBackpressureTest.java   |   2 +-
 .../apache/storm/utils/DisruptorQueueTest.java  |   4 +-
 27 files changed, 1111 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index f89211b..b01e0b7 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -293,3 +293,28 @@ storm.daemon.metrics.reporter.plugins:
 
 # configuration of cluster metrics consumer
 storm.cluster.metrics.consumer.publish.interval.secs: 60
+
+
+storm.metrics.reporters:
+  # Graphite Reporter
+  - class: "org.apache.storm.metrics2.reporters.GraphiteStormReporter"
+    daemons:
+        - "supervisor"
+        - "nimbus"
+        - "worker"
+    report.period: 60
+    report.period.units: "SECONDS"
+    graphite.host: "localhost"
+    graphite.port: 2003
+
+  # Console Reporter
+  - class: "org.apache.storm.metrics2.reporters.ConsoleStormReporter"
+    daemons:
+        - "worker"
+    report.period: 10
+    report.period.units: "SECONDS"
+
+    #TODO: not funtional, but you get the idea
+    filters:
+        "org.apache.storm.metrics2.filters.RegexFilter":
+            expression: ".*my_component.*emitted.*"

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
index 26e0430..9284b52 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
@@ -79,7 +79,7 @@ public class ExclamationTopology {
 
       LocalCluster cluster = new LocalCluster();
       cluster.submitTopology("test", conf, builder.createTopology());
-      Utils.sleep(10000);
+      Utils.sleep(100000);
       cluster.killTopology("test");
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java
new file mode 100644
index 0000000..f05b521
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java
@@ -0,0 +1,121 @@
+package org.apache.storm.starter;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+
+
+public class ReliableWordCount {
+    public static class RandomSentenceSpout extends BaseRichSpout {
+        private static final Logger LOG = LoggerFactory.getLogger(RandomSentenceSpout.class);
+
+        SpoutOutputCollector _collector;
+        Random _rand;
+
+
+        @Override
+        public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+            _collector = collector;
+            _rand = new Random();
+        }
+
+        @Override
+        public void nextTuple() {
+            Utils.sleep(10);
+            String[] sentences = new String[]{sentence("the cow jumped over the moon"), sentence("an apple a day keeps the doctor away"),
+                    sentence("four score and seven years ago"), sentence("snow white and the seven dwarfs"), sentence("i am at two with nature")};
+            final String sentence = sentences[_rand.nextInt(sentences.length)];
+
+            _collector.emit(new Values(sentence), UUID.randomUUID());
+        }
+
+        protected String sentence(String input) {
+            return input;
+        }
+
+        @Override
+        public void ack(Object id) {
+        }
+
+        @Override
+        public void fail(Object id) {
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word"));
+        }
+    }
+
+
+    public static class SplitSentence extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String sentence = tuple.getString(0);
+            for (String word: sentence.split("\\s+")) {
+                collector.emit(new Values(word, 1));
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
+    }
+
+    public static class WordCount extends BaseBasicBolt {
+        Map<String, Integer> counts = new HashMap<String, Integer>();
+
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String word = tuple.getString(0);
+            Integer count = counts.get(word);
+            if (count == null)
+                count = 0;
+            count++;
+            counts.put(word, count);
+            collector.emit(new Values(word, count));
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        TopologyBuilder builder = new TopologyBuilder();
+
+        builder.setSpout("spout", new RandomSentenceSpout(), 4);
+
+        builder.setBolt("split", new SplitSentence(), 4).shuffleGrouping("spout");
+        builder.setBolt("count", new WordCount(), 4).fieldsGrouping("split", new Fields("word"));
+
+        Config conf = new Config();
+        conf.setMaxTaskParallelism(3);
+
+        LocalCluster cluster = new LocalCluster();
+        cluster.submitTopology("word-count", conf, builder.createTopology());
+
+        Thread.sleep(600000);
+
+        cluster.shutdown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
index e4a5711..0611894 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
@@ -98,7 +98,7 @@ public class WordCountTopology {
       LocalCluster cluster = new LocalCluster();
       cluster.submitTopology("word-count", conf, builder.createTopology());
 
-      Thread.sleep(10000);
+      Thread.sleep(60000);
 
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/external/storm-autocreds/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-autocreds/pom.xml b/external/storm-autocreds/pom.xml
index ab654a3..c05b620 100644
--- a/external/storm-autocreds/pom.xml
+++ b/external/storm-autocreds/pom.xml
@@ -15,9 +15,7 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
index 2008a3e..087aec5 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
@@ -27,7 +27,7 @@ import java.io.IOException;
 import java.util.Map;
 
 /**
- * This class provides a mechanism to utilize the Confluent Schema Registry (https://github.com/confluentinc/schema-registry)
+ * This class provides a mechanism to utilize the Confluent Schema StormMetricRegistry (https://github.com/confluentinc/schema-registry)
  * for Storm to (de)serialize Avro generic records across a topology.  It assumes the schema registry is up and running
  * completely independent of Storm.
  */

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f53171b..6ecb150 100644
--- a/pom.xml
+++ b/pom.xml
@@ -886,6 +886,16 @@
                 <version>${metrics.version}</version>
             </dependency>
             <dependency>
+                <groupId>io.dropwizard.metrics</groupId>
+                <artifactId>metrics-graphite</artifactId>
+                <version>${metrics.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>io.dropwizard.metrics</groupId>
+                <artifactId>metrics-ganglia</artifactId>
+                <version>${metrics.version}</version>
+            </dependency>
+            <dependency>
                 <groupId>metrics-clojure</groupId>
                 <artifactId>metrics-clojure</artifactId>
                 <version>${metrics-clojure.version}</version>

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index 0497bdc..e10222a 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -281,6 +281,14 @@
             <artifactId>metrics-core</artifactId>
         </dependency>
         <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-graphite</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.dropwizard.metrics</groupId>
+            <artifactId>metrics-ganglia</artifactId>
+        </dependency>
+        <dependency>
             <groupId>metrics-clojure</groupId>
             <artifactId>metrics-clojure</artifactId>
         </dependency>
@@ -526,7 +534,6 @@
                             <include>org.clojure:tools.namespace</include>
                             <include>cheshire:cheshire</include>
                             <include>org.clojure:core.incubator</include>
-                            <include>io.dropwizard.metrics:*</include>
                             <include>metrics-clojure:*</include>
                         </includes>
                     </artifactSet>
@@ -700,10 +707,10 @@
                           <pattern>org.eclipse.jetty</pattern>
                           <shadedPattern>org.apache.storm.shade.org.eclipse.jetty</shadedPattern>
                         </relocation>
-                        <relocation>
-                            <pattern>com.codahale.metrics</pattern>
-                            <shadedPattern>org.apache.storm.shade.com.codahale.metrics</shadedPattern>
-                        </relocation>
+                        <!--<relocation>-->
+                            <!--<pattern>com.codahale.metrics</pattern>-->
+                            <!--<shadedPattern>org.apache.storm.shade.com.codahale.metrics</shadedPattern>-->
+                        <!--</relocation>-->
                         <relocation>
                             <pattern>metrics.core</pattern>
                             <shadedPattern>org.apache.storm.shade.metrics.core</shadedPattern>

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 8126a80..3e5dd20 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -35,6 +35,8 @@
   (:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
   (:import [org.apache.storm Config Constants])
   (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
+  (:import [org.apache.storm.metrics2 StormMetricRegistry])
+  (:import [com.codahale.metrics Meter])
   (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
   (:import [java.util.concurrent ConcurrentLinkedQueue])
   (:require [org.apache.storm [thrift :as thrift]
@@ -231,6 +233,8 @@
                                   (str "executor"  executor-id "-send-queue")
                                   (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
+                                  (.getStormId worker-context)
+                                  (.getThisWorkerPort worker-context)
                                   :producer-type :single-threaded
                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
@@ -275,7 +279,9 @@
                                (log-message "Got interrupted excpetion shutting thread down...")
                                ((:suicide-fn <>))))
      :sampler (mk-stats-sampler storm-conf)
-     :spout-throttling-metrics (if (= executor-type :spout) 
+     :failed-meter (StormMetricRegistry/meter "failed" worker-context component-id)
+     :acked-meter (StormMetricRegistry/meter "acked" worker-context component-id)
+     :spout-throttling-metrics (if (= executor-type :spout)
                                 (builtin-metrics/make-spout-throttling-data)
                                 nil)
      ;; TODO: add in the executor-specific stuff in a :specific... or make a spout-data, bolt-data function?
@@ -429,10 +435,12 @@
 (defn- fail-spout-msg [executor-data task-data msg-id tuple-info time-delta reason id debug?]
   (let [^ISpout spout (:object task-data)
         storm-conf (:storm-conf executor-data)
-        task-id (:task-id task-data)]
+        task-id (:task-id task-data)
+        failed-meter (:failed-meter executor-data)]
     ;;TODO: need to throttle these when there's lots of failures
     (when debug?
       (log-message "SPOUT Failing " id ": " tuple-info " REASON: " reason " MSG-ID: " msg-id))
+    (.mark failed-meter)
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
@@ -440,8 +448,10 @@
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
   (let [^ISpout spout (:object task-data)
-        task-id (:task-id task-data)]
+        task-id (:task-id task-data)
+        acked-meter (:acked-meter executor-data)]
     (when debug? (log-message "SPOUT Acking message " id " " msg-id))
+    (.mark acked-meter)
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
@@ -809,6 +819,7 @@
                          (let [delta (tuple-time-delta! tuple)]
                            (when debug? 
                              (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
+                           (.mark  ^Meter (:acked-meter (:executor-data task-data)))
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when delta
                              (stats/bolt-acked-tuple! executor-stats
@@ -824,6 +835,7 @@
                                debug? (= true (storm-conf TOPOLOGY-DEBUG))]
                            (when debug? 
                              (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
+                           (.mark  ^Meter (:failed-meter (:executor-data task-data)))
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when delta
                              (stats/bolt-failed-tuple! executor-stats

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 1ae9b22..2e4df75 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -23,10 +23,12 @@
   (:import [org.apache.storm.hooks.info SpoutAckInfo SpoutFailInfo
             EmitInfo BoltFailInfo BoltAckInfo])
   (:import [org.apache.storm.task TopologyContext ShellBolt WorkerTopologyContext])
+  (:import [org.apache.storm.metrics2 StormMetricRegistry])
   (:import [org.apache.storm.utils Utils])
   (:import [org.apache.storm.generated ShellComponent JavaObject])
   (:import [org.apache.storm.spout ShellSpout])
   (:import [java.util Collection List ArrayList])
+  (:import [com.codahale.metrics Meter])
   (:require [org.apache.storm
              [thrift :as thrift]
              [stats :as stats]])
@@ -128,9 +130,11 @@
         stream->component->grouper (:stream->component->grouper executor-data)
         user-context (:user-context task-data)
         executor-stats (:stats executor-data)
-        debug? (= true (storm-conf TOPOLOGY-DEBUG))]
+        debug? (= true (storm-conf TOPOLOGY-DEBUG))
+        ^Meter emitted-meter (StormMetricRegistry/meter "emitted" worker-context component-id)]
         
     (fn ([^Integer out-task-id ^String stream ^List values]
+          (.mark emitted-meter)
           (when debug?
             (log-message "Emitting direct: " out-task-id "; " component-id " " stream " " values))
           (let [target-component (.getComponentId worker-context out-task-id)
@@ -147,6 +151,7 @@
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
+           (.mark emitted-meter)
            (when debug?
              (log-message "Emitting: " component-id " " stream " " values))
            (let [out-tasks (ArrayList.)]

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 6626272..b2810db 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -43,6 +43,7 @@
   (:import [org.apache.logging.log4j Level])
   (:import [org.apache.logging.log4j.core.config LoggerConfig])
   (:import [org.apache.storm.generated LogConfig LogLevelAction])
+  (:import [org.apache.storm.metrics2 StormMetricRegistry])
   (:gen-class))
 
 (defmulti mk-suicide-fn cluster-mode)
@@ -204,17 +205,19 @@
           (transfer-fn serializer tuple-batch)))
       transfer-fn)))
 
-(defn- mk-receive-queue-map [storm-conf executors]
+(defn- mk-receive-queue-map [storm-conf executors storm-id port]
   (->> executors
        ;; TODO: this depends on the type of executor
        (map (fn [e] [e (disruptor/disruptor-queue (str "receive-queue" e)
                                                   (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
+                                                  storm-id port
                                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))]))
        (into {})
        ))
 
+
 (defn- stream->fields [^StormTopology topology component]
   (->> (ThriftTopologyUtils/getComponentCommon topology component)
        .get_streams
@@ -253,9 +256,10 @@
         executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
         transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
+                                                  storm-id port
                                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
-        executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
+        executor-receive-queue-map (mk-receive-queue-map storm-conf executors storm-id port)
 
         receive-queue-map (->> executor-receive-queue-map
                                (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue])))
@@ -595,7 +599,7 @@
       (spit (worker-artifacts-pid-path conf storm-id port) pid)))
 
   (declare establish-log-setting-callback)
-
+  (StormMetricRegistry/start conf DaemonType/WORKER)
   ;; start out with empty list of timeouts 
   (def latest-log-config (atom {}))
   (def original-log-levels (atom {}))
@@ -689,6 +693,8 @@
 
                     (close-resources worker)
 
+                    (StormMetricRegistry/stop)
+
                     (log-message "Trigger any worker shutdown hooks")
                     (run-worker-shutdown-hooks worker)
 

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/clj/org/apache/storm/disruptor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/disruptor.clj b/storm-core/src/clj/org/apache/storm/disruptor.clj
index 1546b3f..73a9d84 100644
--- a/storm-core/src/clj/org/apache/storm/disruptor.clj
+++ b/storm-core/src/clj/org/apache/storm/disruptor.clj
@@ -16,6 +16,7 @@
 
 (ns org.apache.storm.disruptor
   (:import [org.apache.storm.utils DisruptorQueue WorkerBackpressureCallback DisruptorBackpressureCallback])
+  (:import [org.apache.storm.task WorkerTopologyContext])
   (:import [com.lmax.disruptor.dsl ProducerType])
   (:require [clojure [string :as str]])
   (:require [clojure [set :as set]])
@@ -27,10 +28,10 @@
    :single-threaded ProducerType/SINGLE})
 
 (defnk disruptor-queue
-  [^String queue-name buffer-size timeout :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
+  [^String queue-name buffer-size timeout ^String storm-id ^Integer worker-port :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
   (DisruptorQueue. queue-name
                    (PRODUCER-TYPE producer-type) buffer-size
-                   timeout batch-size batch-timeout))
+                   timeout batch-size batch-timeout storm-id worker-port))
 
 (defn clojure-handler
   [afn]

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java
index 43df951..c547530 100644
--- a/storm-core/src/jvm/org/apache/storm/Config.java
+++ b/storm-core/src/jvm/org/apache/storm/Config.java
@@ -139,6 +139,9 @@ public class Config extends HashMap<String, Object> {
     @isString
     public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate";
 
+    @isType(type=List.class)
+    public static final String STORM_METRICS_REPORTERS = "storm.metrics.reporters";
+
     /**
      * A list of daemon metrics  reporter plugin class names.
      * These plugins must implement {@link org.apache.storm.daemon.metrics.reporters.PreparableReporter} interface.

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java b/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
new file mode 100644
index 0000000..994a965
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2;
+
+import org.apache.storm.utils.DisruptorQueue;
+
+public class DisruptorMetrics {
+    private SimpleGauge<Long> capacity;
+    private SimpleGauge<Long> population;
+    private SimpleGauge<Long> writePosition;
+    private SimpleGauge<Long> readPosition;
+    private SimpleGauge<Double> arrivalRate; // TODO: Change to meter
+    private SimpleGauge<Double> sojournTime;
+    private SimpleGauge<Long> overflow;
+    private SimpleGauge<Float> pctFull;
+
+
+    DisruptorMetrics(SimpleGauge<Long> capacity,
+                    SimpleGauge<Long> population,
+                    SimpleGauge<Long> writePosition,
+                    SimpleGauge<Long> readPosition,
+                    SimpleGauge<Double> arrivalRate,
+                    SimpleGauge<Double> sojournTime,
+                    SimpleGauge<Long> overflow,
+                    SimpleGauge<Float> pctFull) {
+        this.capacity = capacity;
+        this.population = population;
+        this.writePosition = writePosition;
+        this.readPosition = readPosition;
+        this.arrivalRate = arrivalRate;
+        this.sojournTime = sojournTime;
+        this.overflow = overflow;
+        this.pctFull = pctFull;
+    }
+
+    public void setCapacity(Long capacity) {
+        this.capacity.set(capacity);
+    }
+
+    public void setPopulation(Long population) {
+        this.population.set(population);
+    }
+
+    public void setWritePosition(Long writePosition) {
+        this.writePosition.set(writePosition);
+    }
+
+    public void setReadPosition(Long readPosition) {
+        this.readPosition.set(readPosition);
+    }
+
+    public void setArrivalRate(Double arrivalRate) {
+        this.arrivalRate.set(arrivalRate);
+    }
+
+    public void setSojournTime(Double soujournTime) {
+        this.sojournTime.set(soujournTime);
+    }
+
+    public void setOverflow(Long overflow) {
+        this.overflow.set(overflow);
+    }
+
+    public void setPercentFull(Float pctFull){
+        this.pctFull.set(pctFull);
+    }
+
+    public void set(DisruptorQueue.QueueMetrics metrics){
+        this.capacity.set(metrics.capacity());
+        this.population.set(metrics.population());
+        this.writePosition.set(metrics.writePos());
+        this.readPosition.set(metrics.readPos());
+        this.arrivalRate.set(metrics.arrivalRate());
+        this.sojournTime.set(metrics.sojournTime());
+        this.overflow.set(metrics.overflow());
+        this.pctFull.set(metrics.pctFull());
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/SimpleGauge.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/SimpleGauge.java b/storm-core/src/jvm/org/apache/storm/metrics2/SimpleGauge.java
new file mode 100644
index 0000000..5240f26
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/SimpleGauge.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2;
+
+
+import com.codahale.metrics.Gauge;
+
+public class SimpleGauge<T> implements Gauge<T> {
+    private T value;
+
+    public SimpleGauge(T value){
+        this.value = value;
+    }
+
+    @Override
+    public T getValue() {
+        return this.value;
+    }
+
+    public void set(T value){
+        this.value = value;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
new file mode 100644
index 0000000..ced1233
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2;
+
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.storm.Config;
+import org.apache.storm.cluster.DaemonType;
+import org.apache.storm.metrics2.reporters.StormReporter;
+import org.apache.storm.task.WorkerTopologyContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+
+public class StormMetricRegistry {
+
+    private static final Logger LOG = LoggerFactory.getLogger(StormMetricRegistry.class);
+
+    private static final MetricRegistry REGISTRY = new MetricRegistry();
+
+    private static final List<StormReporter> REPORTERS = new ArrayList<>();
+
+    private static String hostName = null;
+
+    public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, Integer port){
+        SimpleGauge<T> gauge = new SimpleGauge<>(initialValue);
+        String metricName = String.format("storm.worker.%s.%s-%s", topologyId, port, name);
+        if(REGISTRY.getGauges().containsKey(metricName)){
+            return (SimpleGauge)REGISTRY.getGauges().get(metricName);
+        } else {
+            return REGISTRY.register(metricName, gauge);
+        }
+    }
+
+    public static DisruptorMetrics disruptorMetrics(String name, String topologyId, Integer port){
+        return new DisruptorMetrics(
+                StormMetricRegistry.gauge(0L, name + "-capacity", topologyId, port),
+                StormMetricRegistry.gauge(0L, name + "-population", topologyId, port),
+                StormMetricRegistry.gauge(0L, name + "-write-position", topologyId, port),
+                StormMetricRegistry.gauge(0L, name + "-read-position", topologyId, port),
+                StormMetricRegistry.gauge(0.0, name + "-arrival-rate", topologyId, port),
+                StormMetricRegistry.gauge(0.0, name + "-sojourn-time-ms", topologyId, port),
+                StormMetricRegistry.gauge(0L, name + "-overflow", topologyId, port),
+                StormMetricRegistry.gauge(0.0F, name + "-percent-full", topologyId, port)
+        );
+    }
+
+    public static Meter meter(String name, WorkerTopologyContext context, String componentId){
+        // storm.worker.{topology}.{host}.{port}
+        String metricName = String.format("storm.worker.%s.%s.%s.%s-%s", context.getStormId(), hostName,
+                componentId, context.getThisWorkerPort(), name);
+        return REGISTRY.meter(metricName);
+    }
+
+    public static void start(Map<String, Object> stormConfig, DaemonType type){
+        String localHost = (String)stormConfig.get(Config.STORM_LOCAL_HOSTNAME);
+        if(localHost != null){
+            hostName = localHost;
+        } else {
+            try {
+                hostName = InetAddress.getLocalHost().getCanonicalHostName();
+            } catch (UnknownHostException e) {
+                 LOG.warn("Unable to determine hostname while starting the metrics system. Hostname ill be reported" +
+                         " as 'localhost'.");
+            }
+        }
+
+        LOG.info("Starting metrics reporters...");
+        List<Map<String, Object>> reporterList = (List<Map<String, Object>>)stormConfig.get(Config.STORM_METRICS_REPORTERS);
+        for(Map<String, Object> reporterConfig : reporterList){
+            // only start those requested
+            List<String> daemons = (List<String>)reporterConfig.get("daemons");
+            for(String daemon : daemons){
+                if(DaemonType.valueOf(daemon.toUpperCase()) == type){
+                    startReporter(stormConfig, reporterConfig);
+                }
+            }
+        }
+    }
+
+    public static MetricRegistry registtry(){
+        return REGISTRY;
+    }
+
+    private static void startReporter(Map<String, Object> stormConfig, Map<String, Object> reporterConfig){
+        String clazz = (String)reporterConfig.get("class");
+        StormReporter reporter = null;
+        LOG.info("Attempting to instantiate reporter class: {}", clazz);
+        try{
+            reporter = instantiate(clazz);
+        } catch(Exception e){
+            LOG.warn("Unable to instantiate metrics reporter class: {}. Will skip this reporter.", clazz, e);
+        }
+        if(reporter != null){
+            reporter.prepare(REGISTRY, stormConfig, reporterConfig);
+            reporter.start();
+            REPORTERS.add(reporter);
+        }
+
+    }
+
+    private static StormReporter instantiate(String klass) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+        Class<?> c = Class.forName(klass);
+        return  (StormReporter) c.newInstance();
+    }
+
+    public static void stop(){
+        for(StormReporter sr : REPORTERS){
+            sr.stop();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
new file mode 100644
index 0000000..5322bf8
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
@@ -0,0 +1,63 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.ConsoleReporter;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class ConsoleStormReporter extends SheduledStormReporter<ConsoleReporter> {
+    private final static Logger LOG = LoggerFactory.getLogger(ConsoleStormReporter.class);
+
+    @Override
+    public void prepare(MetricRegistry registry, Map stormConf, Map reporterConf) {
+        LOG.debug("Preparing ConsoleReporter");
+        ConsoleReporter.Builder builder = ConsoleReporter.forRegistry(registry);
+
+        builder.outputTo(System.out);
+        Locale locale = MetricsUtils.getMetricsReporterLocale(stormConf);
+        if (locale != null) {
+            builder.formattedFor(locale);
+        }
+
+        TimeUnit rateUnit = MetricsUtils.getMetricsRateUnit(stormConf);
+        if (rateUnit != null) {
+            builder.convertRatesTo(rateUnit);
+        }
+
+        TimeUnit durationUnit = MetricsUtils.getMetricsDurationUnit(stormConf);
+        if (durationUnit != null) {
+            builder.convertDurationsTo(durationUnit);
+        }
+
+        //defaults to 10
+        reportingPeriod = getReportPeriod(reporterConf);
+
+        //defaults to seconds
+        reportingPeriodUnit = getReportPeriodUnit(reporterConf);
+
+        reporter = builder.build();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
new file mode 100644
index 0000000..4225b7c
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
@@ -0,0 +1,93 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.CsvReporter;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.utils.ConfigUtils;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class CsvStormReporter extends SheduledStormReporter<CsvReporter> {
+    private final static Logger LOG = LoggerFactory.getLogger(CsvStormReporter.class);
+
+    public static final String CSV_LOG_DIR = "csv.log.dir";
+
+    @Override
+    public void prepare(MetricRegistry metricsRegistry, Map stormConf, Map reporterConf) {
+        LOG.debug("Preparing...");
+        CsvReporter.Builder builder = CsvReporter.forRegistry(metricsRegistry);
+
+        Locale locale = MetricsUtils.getMetricsReporterLocale(reporterConf);
+        if (locale != null) {
+            builder.formatFor(locale);
+        }
+
+        TimeUnit rateUnit = MetricsUtils.getMetricsRateUnit(reporterConf);
+        if (rateUnit != null) {
+            builder.convertRatesTo(rateUnit);
+        }
+
+        TimeUnit durationUnit = MetricsUtils.getMetricsDurationUnit(reporterConf);
+        if (durationUnit != null) {
+            builder.convertDurationsTo(durationUnit);
+        }
+
+        //TODO: expose some simple MetricFilters 
+
+        //defaults to 10
+        reportingPeriod = getReportPeriod(reporterConf);
+
+        //defaults to seconds
+        reportingPeriodUnit = getReportPeriodUnit(reporterConf);
+
+        File csvMetricsDir = getCsvLogDir(stormConf, reporterConf);
+        reporter = builder.build(csvMetricsDir);
+    }
+
+
+    private static File getCsvLogDir(Map stormConf, Map reporterConf) {
+        String csvMetricsLogDirectory = Utils.getString(reporterConf.get(CSV_LOG_DIR), null);
+        if (csvMetricsLogDirectory == null) {
+            csvMetricsLogDirectory = ConfigUtils.absoluteStormLocalDir(stormConf);
+            csvMetricsLogDirectory = csvMetricsLogDirectory + ConfigUtils.FILE_SEPARATOR + "csvmetrics";
+        }
+        File csvMetricsDir = new File(csvMetricsLogDirectory);
+        validateCreateOutputDir(csvMetricsDir);
+        return csvMetricsDir;
+    }
+
+    private static void validateCreateOutputDir(File dir) {
+        if (!dir.exists()) {
+            dir.mkdirs();
+        }
+        if (!dir.canWrite()) {
+            throw new IllegalStateException(dir.getName() + " does not have write permissions.");
+        }
+        if (!dir.isDirectory()) {
+            throw new IllegalStateException(dir.getName() + " is not a directory.");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
new file mode 100644
index 0000000..d8d0269
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
@@ -0,0 +1,133 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.ganglia.GangliaReporter;
+import com.codahale.metrics.MetricRegistry;
+import info.ganglia.gmetric4j.gmetric.GMetric;
+import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class GangliaStormReporter extends SheduledStormReporter<GangliaReporter> {
+    private final static Logger LOG = LoggerFactory.getLogger(GangliaStormReporter.class);
+
+    public static final String GANGLIA_HOST = "ganglia.host";
+    public static final String GANGLIA_PORT = "ganglia.port";
+    public static final String GANGLIA_PREFIXED_WITH = "ganglia.prefixed.with";
+    public static final String GANGLIA_DMAX = "ganglia.dmax";
+    public static final String GANGLIA_TMAX = "ganglia.tmax";
+    public static final String GANGLIA_UDP_ADDRESSING_MODE = "ganglia.udp.addressing.mode";
+    public static final String GANGLIA_RATE_UNIT = "ganglia.rate.unit";
+    public static final String GANGLIA_DURATION_UNIT = "ganglia.duration.unit";
+    public static final String GANGLIA_TTL = "ganglia.ttl";
+    public static final String GANGLIA_UDP_GROUP = "ganglia.udp.group";
+
+    @Override
+    public void prepare(MetricRegistry metricsRegistry, Map stormConf, Map reporterConf) {
+        LOG.debug("Preparing...");
+        GangliaReporter.Builder builder = GangliaReporter.forRegistry(metricsRegistry);
+
+        TimeUnit durationUnit = MetricsUtils.getMetricsDurationUnit(reporterConf);
+        if (durationUnit != null) {
+            builder.convertDurationsTo(durationUnit);
+        }
+
+        TimeUnit rateUnit = MetricsUtils.getMetricsRateUnit(reporterConf);
+        if (rateUnit != null) {
+            builder.convertRatesTo(rateUnit);
+        }
+
+        //TODO: expose some simple MetricFilters 
+        String prefix = getMetricsPrefixedWith(reporterConf);
+        if (prefix != null) {
+            builder.prefixedWith(prefix);
+        }
+
+        Integer dmax = getGangliaDMax(reporterConf);
+        if (prefix != null) {
+            builder.withDMax(dmax);
+        }
+
+        Integer tmax = getGangliaTMax(reporterConf);
+        if (prefix != null) {
+            builder.withTMax(tmax);
+        }
+
+        //defaults to 10
+        reportingPeriod = getReportPeriod(reporterConf);
+
+        //defaults to seconds
+        reportingPeriodUnit = getReportPeriodUnit(reporterConf);
+
+        // Not exposed:
+        // * withClock(Clock)
+
+        String group = getMetricsTargetUDPGroup(reporterConf);
+        Integer port = getMetricsTargetPort(reporterConf);
+        String udpAddressingMode = getMetricsTargetUDPAddressingMode(reporterConf);
+        Integer ttl = getMetricsTargetTtl(reporterConf);
+
+        GMetric.UDPAddressingMode mode = udpAddressingMode.equalsIgnoreCase("multicast") ?
+                GMetric.UDPAddressingMode.MULTICAST : GMetric.UDPAddressingMode.UNICAST;
+
+        try {
+            GMetric sender = new GMetric(group, port, mode, ttl);
+            reporter = builder.build(sender);
+        }catch (IOException ioe){
+            //TODO
+            LOG.error("Exception in GangliaReporter config", ioe);
+        }
+    }
+
+
+    public static String getMetricsTargetUDPGroup(Map reporterConf) {
+        return Utils.getString(reporterConf.get(GANGLIA_UDP_GROUP), null);
+    }
+
+    public static String getMetricsTargetUDPAddressingMode(Map reporterConf) {
+        return Utils.getString(reporterConf.get(GANGLIA_UDP_ADDRESSING_MODE), null);
+    }
+
+    public static Integer getMetricsTargetTtl(Map reporterConf) {
+        return Utils.getInt(reporterConf.get(GANGLIA_TTL), null);
+    }
+
+    public static Integer getGangliaDMax(Map reporterConf) {
+        return Utils.getInt(reporterConf.get(GANGLIA_DMAX), null);
+    }
+
+    public static Integer getGangliaTMax(Map reporterConf) {
+        return Utils.getInt(reporterConf.get(GANGLIA_TMAX), null);
+    }
+
+
+    private static Integer getMetricsTargetPort(Map reporterConf) {
+        return Utils.getInt(reporterConf.get(GANGLIA_PORT), null);
+    }
+
+    private static String getMetricsPrefixedWith(Map reporterConf) {
+        return Utils.getString(reporterConf.get(GANGLIA_PREFIXED_WITH), null);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
new file mode 100644
index 0000000..7a2b31b
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
@@ -0,0 +1,100 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.graphite.GraphiteReporter;
+import com.codahale.metrics.graphite.GraphiteSender;
+import com.codahale.metrics.graphite.GraphiteUDP;
+import com.codahale.metrics.graphite.Graphite;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class GraphiteStormReporter extends SheduledStormReporter<GraphiteReporter> {
+    private final static Logger LOG = LoggerFactory.getLogger(GraphiteStormReporter.class);
+
+    public static final String GRAPHITE_PREFIXED_WITH = "graphite.prefixed.with";
+    public static final String GRAPHITE_HOST = "graphite.host";
+    public static final String GRAPHITE_PORT = "graphite.port";
+    public static final String GRAPHITE_TRANSPORT = "graphite.transport";
+
+    @Override
+    public void prepare(MetricRegistry metricsRegistry, Map stormConf, Map reporterConf) {
+        LOG.debug("Preparing...");
+        GraphiteReporter.Builder builder = GraphiteReporter.forRegistry(metricsRegistry);
+
+        TimeUnit durationUnit = MetricsUtils.getMetricsDurationUnit(reporterConf);
+        if (durationUnit != null) {
+            builder.convertDurationsTo(durationUnit);
+        }
+
+        TimeUnit rateUnit = MetricsUtils.getMetricsRateUnit(reporterConf);
+        if (rateUnit != null) {
+            builder.convertRatesTo(rateUnit);
+        }
+
+        //TODO: expose some simple MetricFilters 
+        String prefix = getMetricsPrefixedWith(reporterConf);
+        if (prefix != null) {
+            builder.prefixedWith(prefix);
+        }
+
+        //defaults to 10
+        reportingPeriod = getReportPeriod(reporterConf);
+
+        //defaults to seconds
+        reportingPeriodUnit = getReportPeriodUnit(reporterConf);
+
+        // Not exposed:
+        // * withClock(Clock)
+
+        String host = getMetricsTargetHost(reporterConf);
+        Integer port = getMetricsTargetPort(reporterConf);
+        String transport = getMetricsTargetTransport(reporterConf);
+        GraphiteSender sender = null;
+        //TODO: error checking
+        if (transport.equalsIgnoreCase("udp")) {
+            sender = new GraphiteUDP(host, port);
+        } else {
+            //TODO: pickled support
+            sender = new Graphite(host, port);
+        }
+        reporter = builder.build(sender);
+    }
+
+    private static String getMetricsPrefixedWith(Map reporterConf) {
+        return Utils.getString(reporterConf.get(GRAPHITE_PREFIXED_WITH), null);
+    }
+
+    private static String getMetricsTargetHost(Map reporterConf) {
+        return Utils.getString(reporterConf.get(GRAPHITE_HOST), null);
+    }
+
+    private static Integer getMetricsTargetPort(Map reporterConf) {
+        return Utils.getInt(reporterConf.get(GRAPHITE_PORT), null);
+    }
+
+    private static String getMetricsTargetTransport(Map reporterConf) {
+        return Utils.getString(reporterConf.get(GRAPHITE_TRANSPORT), "tcp");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
new file mode 100644
index 0000000..7ac6cde
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
@@ -0,0 +1,88 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class JmxStormReporter implements StormReporter<JmxReporter> {
+    private final static Logger LOG = LoggerFactory.getLogger(JmxStormReporter.class);
+    public static final String JMX_DOMAIN = "jmx.domain";
+    JmxReporter reporter = null;
+
+    @Override
+    public void prepare(MetricRegistry metricsRegistry, Map<String, Object> stormConf, Map<String, Object> reporterConf) {
+        LOG.info("Preparing...");
+        JmxReporter.Builder builder = JmxReporter.forRegistry(metricsRegistry);
+
+        TimeUnit durationUnit = MetricsUtils.getMetricsDurationUnit(reporterConf);
+        if (durationUnit != null) {
+            builder.convertDurationsTo(durationUnit);
+        }
+
+        TimeUnit rateUnit = MetricsUtils.getMetricsRateUnit(reporterConf);
+        if (rateUnit != null) {
+            builder.convertRatesTo(rateUnit);
+        }
+
+        String domain = getMetricsJMXDomain(reporterConf);
+        if (domain != null) {
+            builder.inDomain(domain);
+        }
+
+        // TODO: expose some simple MetricFilters
+        // other builder functions not exposed:
+        //  * createsObjectNamesWith(ObjectNameFactory onFactory) 
+        //  * registerWith (MBeanServer)
+        //  * specificDurationUnits (Map<String,TimeUnit> specificDurationUnits)
+        //  * specificRateUnits(Map<String,TimeUnit> specificRateUnits)
+
+        reporter = builder.build();
+    }
+
+    public static String getMetricsJMXDomain(Map reporterConf) {
+        return Utils.getString(reporterConf, JMX_DOMAIN);
+    }
+
+    @Override
+    public void start() {
+        if (reporter != null) {
+            LOG.debug("Starting...");
+            reporter.start();
+        } else {
+            throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName());
+        }
+    }
+
+    @Override
+    public void stop() {
+        if (reporter != null) {
+            LOG.debug("Stopping...");
+            reporter.stop();
+        } else {
+            throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java
new file mode 100644
index 0000000..1b1e7a0
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.ScheduledReporter;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public abstract class SheduledStormReporter<T extends ScheduledReporter> implements StormReporter{
+    private static final Logger LOG = LoggerFactory.getLogger(SheduledStormReporter.class);
+    protected ScheduledReporter reporter;
+    long reportingPeriod;
+    TimeUnit reportingPeriodUnit;
+
+    @Override
+    public void start() {
+        if (reporter != null) {
+            LOG.debug("Starting...");
+            reporter.start(reportingPeriod, reportingPeriodUnit);
+        } else {
+            throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName());
+        }
+    }
+
+    @Override
+    public void stop() {
+        if (reporter != null) {
+            LOG.debug("Stopping...");
+            reporter.stop();
+        } else {
+            throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName());
+        }
+    }
+
+
+    static TimeUnit getReportPeriodUnit(Map<String, Object> reporterConf) {
+        TimeUnit unit = getTimeUnitForConfig(reporterConf, REPORT_PERIOD_UNITS);
+        return unit == null ? TimeUnit.SECONDS : unit;
+    }
+
+    private static TimeUnit getTimeUnitForConfig(Map reporterConf, String configName) {
+        String rateUnitString = Utils.getString(reporterConf.get(configName), null);
+        if (rateUnitString != null) {
+            return TimeUnit.valueOf(rateUnitString);
+        }
+        return null;
+    }
+
+    static long getReportPeriod(Map reporterConf) {
+        return Utils.getInt(reporterConf.get(REPORT_PERIOD), 10).longValue();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java
new file mode 100644
index 0000000..c36e44e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Reporter;
+
+import java.util.Map;
+
+public interface StormReporter<T extends Reporter> {
+    String REPORT_PERIOD = "report.period";
+    String REPORT_PERIOD_UNITS = "report.period.units";
+
+    void prepare(MetricRegistry metricsRegistry, Map<String, Object> conf, Map<String, Object> reporterConf);
+    void start();
+    void stop();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
index 91cbee9..080eb9a 100644
--- a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
+++ b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
@@ -17,6 +17,7 @@
  */
 package org.apache.storm.task;
 
+import com.codahale.metrics.*;
 import org.apache.storm.generated.GlobalStreamId;
 import org.apache.storm.generated.Grouping;
 import org.apache.storm.generated.StormTopology;
@@ -26,6 +27,7 @@ import org.apache.storm.metric.api.IReducer;
 import org.apache.storm.metric.api.ICombiner;
 import org.apache.storm.metric.api.ReducedMetric;
 import org.apache.storm.metric.api.CombinedMetric;
+import org.apache.storm.metrics2.StormMetricRegistry;
 import org.apache.storm.state.ISubscribedState;
 import org.apache.storm.tuple.Fields;
 import org.apache.storm.utils.Utils;
@@ -386,4 +388,28 @@ public class TopologyContext extends WorkerTopologyContext implements IMetricsCo
     public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) {
         return registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs);
     }
+
+    public Timer registerTimer(String name){
+        return StormMetricRegistry.registtry().timer(metricName(name));
+    }
+
+    public Histogram registerHistogram(String name){
+        return StormMetricRegistry.registtry().histogram(metricName(name));
+    }
+
+    public Meter registerMeter(String name){
+        return StormMetricRegistry.registtry().meter(metricName(name));
+    }
+
+    public Counter registerCounter(String name){
+        return StormMetricRegistry.registtry().counter(metricName(name));
+    }
+
+    public Gauge registerGauge(String name, Gauge gauge){
+        return StormMetricRegistry.registtry().register(metricName(name), gauge);
+    }
+
+    private String metricName(String name){
+        return String.format("storm.topology.%s.%s.%s-%s", getStormId(), getThisComponentId(), getThisWorkerPort(), name);
+    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
index fe90240..35bc83f 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
@@ -34,6 +34,9 @@ import com.lmax.disruptor.dsl.ProducerType;
 import org.apache.storm.Config;
 import org.apache.storm.metric.api.IStatefulObject;
 import org.apache.storm.metric.internal.RateTracker;
+import org.apache.storm.metrics2.DisruptorMetrics;
+import org.apache.storm.metrics2.StormMetricRegistry;
+import org.apache.storm.task.WorkerTopologyContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,6 +67,7 @@ public class DisruptorQueue implements IStatefulObject {
     private static final String PREFIX = "disruptor-";
     private static final FlusherPool FLUSHER = new FlusherPool();
     
+    private static final Timer METRICS_TIMER = new Timer("disruptor-metrics-timer", true);
     private static int getNumFlusherPoolThreads() {
         int numThreads = 100;
         try {
@@ -345,27 +349,31 @@ public class DisruptorQueue implements IStatefulObject {
             return (1.0F * population() / capacity());
         }
 
-        public Object getState() {
-            Map state = new HashMap<String, Object>();
+        public double arrivalRate(){
+            return _rateTracker.reportRate();
+        }
 
+        public double sojournTime(){
             // get readPos then writePos so it's never an under-estimate
             long rp = readPos();
             long wp = writePos();
-
-            final double arrivalRateInSecs = _rateTracker.reportRate();
+            final double arrivalRateInSecs = arrivalRate();
 
             //Assume the queue is stable, in which the arrival rate is equal to the consumption rate.
             // If this assumption does not hold, the calculation of sojourn time should also consider
             // departure rate according to Queuing Theory.
-            final double sojournTime = (wp - rp) / Math.max(arrivalRateInSecs, 0.00001) * 1000.0;
+            return (wp - rp) / Math.max(arrivalRateInSecs, 0.00001) * 1000.0;
+        }
 
+        public Object getState() {
+            Map state = new HashMap<String, Object>();
             state.put("capacity", capacity());
-            state.put("population", wp - rp);
-            state.put("write_pos", wp);
-            state.put("read_pos", rp);
-            state.put("arrival_rate_secs", arrivalRateInSecs);
-            state.put("sojourn_time_ms", sojournTime); //element sojourn time in milliseconds
-            state.put("overflow", _overflowCount.get());
+            state.put("population", population());
+            state.put("write_pos", writePos());
+            state.put("read_pos", readPos());
+            state.put("arrival_rate_secs", arrivalRate());
+            state.put("sojourn_time_ms", sojournTime()); //element sojourn time in milliseconds
+            state.put("overflow", overflow());
 
             return state;
         }
@@ -385,7 +393,8 @@ public class DisruptorQueue implements IStatefulObject {
     private final int _inputBatchSize;
     private final ConcurrentHashMap<Long, ThreadLocalInserter> _batchers = new ConcurrentHashMap<Long, ThreadLocalInserter>();
     private final Flusher _flusher;
-    private final QueueMetrics _metrics;
+    private final QueueMetrics _metrics; // old metrics API
+    private final DisruptorMetrics _disruptorMetrics;
 
     private String _queueName = "";
     private DisruptorBackpressureCallback _cb = null;
@@ -395,7 +404,7 @@ public class DisruptorQueue implements IStatefulObject {
     private final AtomicLong _overflowCount = new AtomicLong(0);
     private volatile boolean _throttleOn = false;
 
-    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval) {
+    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval, String topologyId, int port) {
         this._queueName = PREFIX + queueName;
         WaitStrategy wait;
         if (readTimeout <= 0) {
@@ -409,12 +418,20 @@ public class DisruptorQueue implements IStatefulObject {
         _barrier = _buffer.newBarrier();
         _buffer.addGatingSequences(_consumer);
         _metrics = new QueueMetrics();
+        _disruptorMetrics = StormMetricRegistry.disruptorMetrics(_queueName, topologyId, port);
         //The batch size can be no larger than half the full queue size.
         //This is mostly to avoid contention issues.
         _inputBatchSize = Math.max(1, Math.min(inputBatchSize, size/2));
 
         _flusher = new Flusher(Math.max(flushInterval, 1), _queueName);
         _flusher.start();
+
+        METRICS_TIMER.schedule(new TimerTask(){
+            @Override
+            public void run() {
+                _disruptorMetrics.set(_metrics);
+            }
+        }, 15000, 15000); // TODO: Configurable interval
     }
 
     public String getName() {

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
index 7072e55..110fe88 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
@@ -105,6 +105,6 @@ public class DisruptorQueueBackpressureTest extends TestCase {
     }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", 1000);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/fa1e59f4/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
index e7ac54e..c834cbb 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
@@ -178,10 +178,10 @@ public class DisruptorQueueTest extends TestCase {
     }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", 1000);
     }
 
     private static DisruptorQueue createQueue(String name, int batchSize, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, batchSize, 1L);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, batchSize, 1L, "test", 1000);
     }
 }


[17/38] storm git commit: Merge branch 'metrics_v2' of https://git-wip-us.apache.org/repos/asf/storm into metrics_v2

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


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

Branch: refs/heads/1.x-branch
Commit: dd977e8d384c49a7b38cdf30d1ce9b7f09055dcd
Parents: a9647d8 00a382b
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Dec 8 15:15:08 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Dec 8 15:15:08 2017 -0500

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    | 22 ++++------
 .../src/clj/org/apache/storm/daemon/task.clj    |  8 ++--
 storm-core/src/clj/org/apache/storm/stats.clj   | 43 ++++++++++++--------
 3 files changed, 39 insertions(+), 34 deletions(-)
----------------------------------------------------------------------



[03/38] storm git commit: address several review comments

Posted by pt...@apache.org.
address several review comments


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

Branch: refs/heads/1.x-branch
Commit: 0be278a4a8ae8aba5a0e86fc54e69b5c044b5377
Parents: 9a8dfb7
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Aug 11 15:29:53 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Aug 11 15:29:53 2017 -0400

----------------------------------------------------------------------
 .../apache/storm/starter/AnchoredWordCount.java | 138 +++++++++++++++++++
 .../apache/storm/starter/ReliableWordCount.java | 121 ----------------
 storm-core/pom.xml                              |   4 -
 .../storm/metrics2/StormMetricRegistry.java     |  18 ++-
 .../org/apache/storm/task/TopologyContext.java  |  10 +-
 5 files changed, 154 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/0be278a4/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java
new file mode 100644
index 0000000..3b22c9f
--- /dev/null
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.starter;
+
+import org.apache.storm.Config;
+import org.apache.storm.LocalCluster;
+import org.apache.storm.spout.SpoutOutputCollector;
+import org.apache.storm.task.TopologyContext;
+import org.apache.storm.topology.BasicOutputCollector;
+import org.apache.storm.topology.OutputFieldsDeclarer;
+import org.apache.storm.topology.TopologyBuilder;
+import org.apache.storm.topology.base.BaseBasicBolt;
+import org.apache.storm.topology.base.BaseRichSpout;
+import org.apache.storm.tuple.Fields;
+import org.apache.storm.tuple.Tuple;
+import org.apache.storm.tuple.Values;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+
+
+public class AnchoredWordCount {
+    public static class RandomSentenceSpout extends BaseRichSpout {
+        private static final Logger LOG = LoggerFactory.getLogger(RandomSentenceSpout.class);
+
+        SpoutOutputCollector _collector;
+        Random _rand;
+
+
+        @Override
+        public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
+            _collector = collector;
+            _rand = new Random();
+        }
+
+        @Override
+        public void nextTuple() {
+            Utils.sleep(10);
+            String[] sentences = new String[]{sentence("the cow jumped over the moon"), sentence("an apple a day keeps the doctor away"),
+                    sentence("four score and seven years ago"), sentence("snow white and the seven dwarfs"), sentence("i am at two with nature")};
+            final String sentence = sentences[_rand.nextInt(sentences.length)];
+
+            _collector.emit(new Values(sentence), UUID.randomUUID());
+        }
+
+        protected String sentence(String input) {
+            return input;
+        }
+
+        @Override
+        public void ack(Object id) {
+        }
+
+        @Override
+        public void fail(Object id) {
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word"));
+        }
+    }
+
+
+    public static class SplitSentence extends BaseBasicBolt {
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String sentence = tuple.getString(0);
+            for (String word: sentence.split("\\s+")) {
+                collector.emit(new Values(word, 1));
+            }
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
+    }
+
+    public static class WordCount extends BaseBasicBolt {
+        Map<String, Integer> counts = new HashMap<String, Integer>();
+
+        @Override
+        public void execute(Tuple tuple, BasicOutputCollector collector) {
+            String word = tuple.getString(0);
+            Integer count = counts.get(word);
+            if (count == null)
+                count = 0;
+            count++;
+            counts.put(word, count);
+            collector.emit(new Values(word, count));
+        }
+
+        @Override
+        public void declareOutputFields(OutputFieldsDeclarer declarer) {
+            declarer.declare(new Fields("word", "count"));
+        }
+    }
+
+    public static void main(String[] args) throws Exception {
+        TopologyBuilder builder = new TopologyBuilder();
+
+        builder.setSpout("spout", new RandomSentenceSpout(), 4);
+
+        builder.setBolt("split", new SplitSentence(), 4).shuffleGrouping("spout");
+        builder.setBolt("count", new WordCount(), 4).fieldsGrouping("split", new Fields("word"));
+
+        Config conf = new Config();
+        conf.setMaxTaskParallelism(3);
+
+        LocalCluster cluster = new LocalCluster();
+        cluster.submitTopology("word-count", conf, builder.createTopology());
+
+        Thread.sleep(600000);
+
+        cluster.shutdown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/0be278a4/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java
deleted file mode 100644
index f05b521..0000000
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ReliableWordCount.java
+++ /dev/null
@@ -1,121 +0,0 @@
-package org.apache.storm.starter;
-
-import org.apache.storm.Config;
-import org.apache.storm.LocalCluster;
-import org.apache.storm.spout.SpoutOutputCollector;
-import org.apache.storm.task.TopologyContext;
-import org.apache.storm.topology.BasicOutputCollector;
-import org.apache.storm.topology.OutputFieldsDeclarer;
-import org.apache.storm.topology.TopologyBuilder;
-import org.apache.storm.topology.base.BaseBasicBolt;
-import org.apache.storm.topology.base.BaseRichSpout;
-import org.apache.storm.tuple.Fields;
-import org.apache.storm.tuple.Tuple;
-import org.apache.storm.tuple.Values;
-import org.apache.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-
-
-public class ReliableWordCount {
-    public static class RandomSentenceSpout extends BaseRichSpout {
-        private static final Logger LOG = LoggerFactory.getLogger(RandomSentenceSpout.class);
-
-        SpoutOutputCollector _collector;
-        Random _rand;
-
-
-        @Override
-        public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
-            _collector = collector;
-            _rand = new Random();
-        }
-
-        @Override
-        public void nextTuple() {
-            Utils.sleep(10);
-            String[] sentences = new String[]{sentence("the cow jumped over the moon"), sentence("an apple a day keeps the doctor away"),
-                    sentence("four score and seven years ago"), sentence("snow white and the seven dwarfs"), sentence("i am at two with nature")};
-            final String sentence = sentences[_rand.nextInt(sentences.length)];
-
-            _collector.emit(new Values(sentence), UUID.randomUUID());
-        }
-
-        protected String sentence(String input) {
-            return input;
-        }
-
-        @Override
-        public void ack(Object id) {
-        }
-
-        @Override
-        public void fail(Object id) {
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("word"));
-        }
-    }
-
-
-    public static class SplitSentence extends BaseBasicBolt {
-        @Override
-        public void execute(Tuple tuple, BasicOutputCollector collector) {
-            String sentence = tuple.getString(0);
-            for (String word: sentence.split("\\s+")) {
-                collector.emit(new Values(word, 1));
-            }
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("word", "count"));
-        }
-    }
-
-    public static class WordCount extends BaseBasicBolt {
-        Map<String, Integer> counts = new HashMap<String, Integer>();
-
-        @Override
-        public void execute(Tuple tuple, BasicOutputCollector collector) {
-            String word = tuple.getString(0);
-            Integer count = counts.get(word);
-            if (count == null)
-                count = 0;
-            count++;
-            counts.put(word, count);
-            collector.emit(new Values(word, count));
-        }
-
-        @Override
-        public void declareOutputFields(OutputFieldsDeclarer declarer) {
-            declarer.declare(new Fields("word", "count"));
-        }
-    }
-
-    public static void main(String[] args) throws Exception {
-        TopologyBuilder builder = new TopologyBuilder();
-
-        builder.setSpout("spout", new RandomSentenceSpout(), 4);
-
-        builder.setBolt("split", new SplitSentence(), 4).shuffleGrouping("spout");
-        builder.setBolt("count", new WordCount(), 4).fieldsGrouping("split", new Fields("word"));
-
-        Config conf = new Config();
-        conf.setMaxTaskParallelism(3);
-
-        LocalCluster cluster = new LocalCluster();
-        cluster.submitTopology("word-count", conf, builder.createTopology());
-
-        Thread.sleep(600000);
-
-        cluster.shutdown();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/0be278a4/storm-core/pom.xml
----------------------------------------------------------------------
diff --git a/storm-core/pom.xml b/storm-core/pom.xml
index e10222a..499a404 100644
--- a/storm-core/pom.xml
+++ b/storm-core/pom.xml
@@ -707,10 +707,6 @@
                           <pattern>org.eclipse.jetty</pattern>
                           <shadedPattern>org.apache.storm.shade.org.eclipse.jetty</shadedPattern>
                         </relocation>
-                        <!--<relocation>-->
-                            <!--<pattern>com.codahale.metrics</pattern>-->
-                            <!--<shadedPattern>org.apache.storm.shade.com.codahale.metrics</shadedPattern>-->
-                        <!--</relocation>-->
                         <relocation>
                             <pattern>metrics.core</pattern>
                             <shadedPattern>org.apache.storm.shade.metrics.core</shadedPattern>

http://git-wip-us.apache.org/repos/asf/storm/blob/0be278a4/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index ced1233..845745f 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -45,9 +45,9 @@ public class StormMetricRegistry {
 
     public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, Integer port){
         SimpleGauge<T> gauge = new SimpleGauge<>(initialValue);
-        String metricName = String.format("storm.worker.%s.%s-%s", topologyId, port, name);
-        if(REGISTRY.getGauges().containsKey(metricName)){
-            return (SimpleGauge)REGISTRY.getGauges().get(metricName);
+        String metricName = metricName(name, topologyId, null, port);
+            if(REGISTRY.getGauges().containsKey(metricName)){
+                return (SimpleGauge)REGISTRY.getGauges().get(metricName);
         } else {
             return REGISTRY.register(metricName, gauge);
         }
@@ -67,9 +67,7 @@ public class StormMetricRegistry {
     }
 
     public static Meter meter(String name, WorkerTopologyContext context, String componentId){
-        // storm.worker.{topology}.{host}.{port}
-        String metricName = String.format("storm.worker.%s.%s.%s.%s-%s", context.getStormId(), hostName,
-                componentId, context.getThisWorkerPort(), name);
+        String metricName = metricName(name, context.getStormId(), componentId, context.getThisWorkerPort());
         return REGISTRY.meter(metricName);
     }
 
@@ -99,7 +97,7 @@ public class StormMetricRegistry {
         }
     }
 
-    public static MetricRegistry registtry(){
+    public static MetricRegistry registry(){
         return REGISTRY;
     }
 
@@ -130,4 +128,10 @@ public class StormMetricRegistry {
             sr.stop();
         }
     }
+
+    public static String metricName(String name, String stormId, String componentId, Integer workerPort){
+        return String.format("storm.worker.%s.%s.%s.%s-%s", stormId, hostName, componentId, workerPort, name);
+    }
+
+
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/0be278a4/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
index 080eb9a..444a8a7 100644
--- a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
+++ b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
@@ -390,23 +390,23 @@ public class TopologyContext extends WorkerTopologyContext implements IMetricsCo
     }
 
     public Timer registerTimer(String name){
-        return StormMetricRegistry.registtry().timer(metricName(name));
+        return StormMetricRegistry.registry().timer(metricName(name));
     }
 
     public Histogram registerHistogram(String name){
-        return StormMetricRegistry.registtry().histogram(metricName(name));
+        return StormMetricRegistry.registry().histogram(metricName(name));
     }
 
     public Meter registerMeter(String name){
-        return StormMetricRegistry.registtry().meter(metricName(name));
+        return StormMetricRegistry.registry().meter(metricName(name));
     }
 
     public Counter registerCounter(String name){
-        return StormMetricRegistry.registtry().counter(metricName(name));
+        return StormMetricRegistry.registry().counter(metricName(name));
     }
 
     public Gauge registerGauge(String name, Gauge gauge){
-        return StormMetricRegistry.registtry().register(metricName(name), gauge);
+        return StormMetricRegistry.registry().register(metricName(name), gauge);
     }
 
     private String metricName(String name){


[28/38] storm git commit: STORM-2153: use taskId in metrics names instead of executorId

Posted by pt...@apache.org.
STORM-2153: use taskId in metrics names instead of executorId


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

Branch: refs/heads/1.x-branch
Commit: 8d53800f14ced3fd630a02dfd9537d5900979562
Parents: 8bf7252
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Jan 10 14:25:50 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Jan 10 14:25:50 2018 -0500

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/executor.clj |  8 ++++----
 storm-core/src/clj/org/apache/storm/daemon/task.clj     |  8 ++++----
 .../org/apache/storm/metrics2/StormMetricRegistry.java  | 12 ++++++------
 3 files changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/8d53800f/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 993add6..e8d23e5 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -442,7 +442,7 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-failed-tuple! (:stats executor-data) (StormMetricRegistry/counter "failed" (:worker-context executor-data) (:component-id executor-data) (pr-str (:executor-id executor-data)) (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-failed-tuple! (:stats executor-data) (StormMetricRegistry/counter "failed" (:worker-context executor-data) (:component-id executor-data) task-id (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
   (let [^ISpout spout (:object task-data)
@@ -451,7 +451,7 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-acked-tuple! (:stats executor-data) (StormMetricRegistry/counter "acked" (:worker-context executor-data) (:component-id executor-data) (pr-str (:executor-id executor-data)) (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-acked-tuple! (:stats executor-data) (StormMetricRegistry/counter "acked" (:worker-context executor-data) (:component-id executor-data) task-id (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)
@@ -820,7 +820,7 @@
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-acked-tuple! executor-stats
-                                                      (StormMetricRegistry/counter "acked" worker-context  (:component-id executor-data) (pr-str (:executor-id executor-data)) (.getSourceStreamId tuple))
+                                                      (StormMetricRegistry/counter "acked" worker-context  (:component-id executor-data) task-id (.getSourceStreamId tuple))
                                                       (.getSourceComponent tuple)
                                                       (.getSourceStreamId tuple)
                                                       delta))))
@@ -836,7 +836,7 @@
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-failed-tuple! executor-stats
-                                                       (StormMetricRegistry/counter "failed" worker-context (:component-id executor-data) (pr-str (:executor-id executor-data)) (.getSourceStreamId tuple))
+                                                       (StormMetricRegistry/counter "failed" worker-context (:component-id executor-data) task-id (.getSourceStreamId tuple))
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)
                                                        delta))))

http://git-wip-us.apache.org/repos/asf/storm/blob/8d53800f/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 7132fc1..9e18331 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -143,9 +143,9 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream)
+              (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id task-id stream) stream)
               (if out-task-id
-                (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream 1)))
+                (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id task-id stream) stream 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
@@ -163,8 +163,8 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream)
-               (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream (count out-tasks)))
+               (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id task-id stream) stream)
+               (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id task-id stream) stream (count out-tasks)))
              out-tasks)))
     ))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/8d53800f/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index aea4539..e1305f9 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -68,13 +68,13 @@ public class StormMetricRegistry {
         );
     }
 
-    public static Meter meter(String name, WorkerTopologyContext context, String componentId, String executorId, String streamId){
-        String metricName = metricName(name, context.getStormId(), componentId, streamId,executorId, context.getThisWorkerPort());
+    public static Meter meter(String name, WorkerTopologyContext context, String componentId, Integer taskId, String streamId){
+        String metricName = metricName(name, context.getStormId(), componentId, streamId,taskId, context.getThisWorkerPort());
         return REGISTRY.meter(metricName);
     }
 
-    public static Counter counter(String name, WorkerTopologyContext context, String componentId, String executorId, String streamId){
-        String metricName = metricName(name, context.getStormId(), componentId, streamId,executorId, context.getThisWorkerPort());
+    public static Counter counter(String name, WorkerTopologyContext context, String componentId, Integer taskId, String streamId){
+        String metricName = metricName(name, context.getStormId(), componentId, streamId,taskId, context.getThisWorkerPort());
         return REGISTRY.counter(metricName);
     }
 
@@ -124,7 +124,7 @@ public class StormMetricRegistry {
         }
     }
 
-    public static String metricName(String name, String stormId, String componentId, String streamId, String executorId, Integer workerPort){
+    public static String metricName(String name, String stormId, String componentId, String streamId, Integer taskId, Integer workerPort){
         StringBuilder sb = new StringBuilder("storm.worker.");
         sb.append(stormId);
         sb.append(".");
@@ -134,7 +134,7 @@ public class StormMetricRegistry {
         sb.append(".");
         sb.append(dotToUnderScore(streamId));
         sb.append(".");
-        sb.append(dotToUnderScore(executorId));
+        sb.append(taskId);
         sb.append(".");
         sb.append(workerPort);
         sb.append("-");


[05/38] storm git commit: implement metric filters

Posted by pt...@apache.org.
implement metric filters


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

Branch: refs/heads/1.x-branch
Commit: b8de0f365a81a8ba3eadec466d777e47dacfcb44
Parents: 6eaa1a8
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Aug 30 16:14:32 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Aug 30 16:14:32 2017 -0400

----------------------------------------------------------------------
 conf/defaults.yaml                              | 46 +++++-----
 .../apache/storm/metrics2/Metrics2Utils.java    | 28 +++++++
 .../storm/metrics2/StormMetricRegistry.java     | 20 ++---
 .../storm/metrics2/filters/RegexFilter.java     | 47 +++++++++++
 .../metrics2/filters/StormMetricsFilter.java    | 32 +++++++
 .../reporters/ConsoleStormReporter.java         |  9 +-
 .../metrics2/reporters/CsvStormReporter.java    |  9 +-
 .../reporters/GangliaStormReporter.java         |  8 +-
 .../reporters/GraphiteStormReporter.java        |  8 +-
 .../metrics2/reporters/JmxStormReporter.java    |  6 +-
 .../reporters/ScheduledStormReporter.java       | 88 ++++++++++++++++++++
 .../reporters/SheduledStormReporter.java        | 71 ----------------
 12 files changed, 259 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index b01e0b7..e51b50c 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -295,26 +295,26 @@ storm.daemon.metrics.reporter.plugins:
 storm.cluster.metrics.consumer.publish.interval.secs: 60
 
 
-storm.metrics.reporters:
-  # Graphite Reporter
-  - class: "org.apache.storm.metrics2.reporters.GraphiteStormReporter"
-    daemons:
-        - "supervisor"
-        - "nimbus"
-        - "worker"
-    report.period: 60
-    report.period.units: "SECONDS"
-    graphite.host: "localhost"
-    graphite.port: 2003
-
-  # Console Reporter
-  - class: "org.apache.storm.metrics2.reporters.ConsoleStormReporter"
-    daemons:
-        - "worker"
-    report.period: 10
-    report.period.units: "SECONDS"
-
-    #TODO: not funtional, but you get the idea
-    filters:
-        "org.apache.storm.metrics2.filters.RegexFilter":
-            expression: ".*my_component.*emitted.*"
+# Metrics v2 configuration (optional)
+#storm.metrics.reporters:
+#  # Graphite Reporter
+#  - class: "org.apache.storm.metrics2.reporters.GraphiteStormReporter"
+#    daemons:
+#        - "supervisor"
+#        - "nimbus"
+#        - "worker"
+#    report.period: 60
+#    report.period.units: "SECONDS"
+#    graphite.host: "localhost"
+#    graphite.port: 2003
+#
+#  # Console Reporter
+#  - class: "org.apache.storm.metrics2.reporters.ConsoleStormReporter"
+#    daemons:
+#        - "worker"
+#    report.period: 10
+#    report.period.units: "SECONDS"
+#
+#    filter:
+#        class: "org.apache.storm.metrics2.filters.RegexFilter"
+#        expression: ".*my_component.*emitted.*"

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java b/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java
new file mode 100644
index 0000000..716b8b7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2;
+
+public class Metrics2Utils {
+    private Metrics2Utils(){}
+
+    public static Object instantiate(String klass) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+        Class<?> c = Class.forName(klass);
+        return  c.newInstance();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 4c975a3..a3b0db9 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -86,12 +86,14 @@ public class StormMetricRegistry {
 
         LOG.info("Starting metrics reporters...");
         List<Map<String, Object>> reporterList = (List<Map<String, Object>>)stormConfig.get(Config.STORM_METRICS_REPORTERS);
-        for(Map<String, Object> reporterConfig : reporterList){
-            // only start those requested
-            List<String> daemons = (List<String>)reporterConfig.get("daemons");
-            for(String daemon : daemons){
-                if(DaemonType.valueOf(daemon.toUpperCase()) == type){
-                    startReporter(stormConfig, reporterConfig);
+        if(reporterList != null && reporterList.size() > 0) {
+            for (Map<String, Object> reporterConfig : reporterList) {
+                // only start those requested
+                List<String> daemons = (List<String>) reporterConfig.get("daemons");
+                for (String daemon : daemons) {
+                    if (DaemonType.valueOf(daemon.toUpperCase()) == type) {
+                        startReporter(stormConfig, reporterConfig);
+                    }
                 }
             }
         }
@@ -106,7 +108,7 @@ public class StormMetricRegistry {
         StormReporter reporter = null;
         LOG.info("Attempting to instantiate reporter class: {}", clazz);
         try{
-            reporter = instantiate(clazz);
+            reporter = (StormReporter)Metrics2Utils.instantiate(clazz);
         } catch(Exception e){
             LOG.warn("Unable to instantiate metrics reporter class: {}. Will skip this reporter.", clazz, e);
         }
@@ -118,10 +120,6 @@ public class StormMetricRegistry {
 
     }
 
-    private static StormReporter instantiate(String klass) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
-        Class<?> c = Class.forName(klass);
-        return  (StormReporter) c.newInstance();
-    }
 
     public static void stop(){
         for(StormReporter sr : REPORTERS){

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/filters/RegexFilter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/filters/RegexFilter.java b/storm-core/src/jvm/org/apache/storm/metrics2/filters/RegexFilter.java
new file mode 100644
index 0000000..e6997c6
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/filters/RegexFilter.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.filters;
+
+import com.codahale.metrics.Metric;
+
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class RegexFilter implements StormMetricsFilter {
+
+    private Pattern pattern;
+
+
+    @Override
+    public void prepare(Map<String, Object> config) {
+        String expression = (String) config.get("expression");
+        if(expression != null){
+            this.pattern = Pattern.compile(expression);
+        } else {
+            throw new IllegalStateException("RegexFilter requires an 'expression' parameter.");
+        }
+    }
+
+    @Override
+    public boolean matches(String name, Metric metric) {
+        Matcher matcher = this.pattern.matcher(name);
+        return matcher.matches();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java b/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java
new file mode 100644
index 0000000..57f7255
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.filters;
+
+import com.codahale.metrics.MetricFilter;
+
+import java.util.Map;
+
+public interface StormMetricsFilter extends MetricFilter {
+
+    /**
+     * Called after the filter is instantiated.
+     * @param config an arbitrary configuration map pulled from the yaml configuration.
+     */
+    void prepare(Map<String, Object> config);
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
index 5322bf8..abb5226 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
@@ -20,6 +20,7 @@ package org.apache.storm.metrics2.reporters;
 import com.codahale.metrics.ConsoleReporter;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.metrics2.filters.StormMetricsFilter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -27,7 +28,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class ConsoleStormReporter extends SheduledStormReporter<ConsoleReporter> {
+public class ConsoleStormReporter extends ScheduledStormReporter<ConsoleReporter> {
     private final static Logger LOG = LoggerFactory.getLogger(ConsoleStormReporter.class);
 
     @Override
@@ -51,6 +52,12 @@ public class ConsoleStormReporter extends SheduledStormReporter<ConsoleReporter>
             builder.convertDurationsTo(durationUnit);
         }
 
+        StormMetricsFilter filter = getMetricsFilter(reporterConf);
+        if(filter != null){
+            builder.filter(filter);
+        }
+
+
         //defaults to 10
         reportingPeriod = getReportPeriod(reporterConf);
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
index 4225b7c..24c6eed 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
@@ -20,6 +20,7 @@ package org.apache.storm.metrics2.reporters;
 import com.codahale.metrics.CsvReporter;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.metrics2.filters.StormMetricsFilter;
 import org.apache.storm.utils.ConfigUtils;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
@@ -30,7 +31,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class CsvStormReporter extends SheduledStormReporter<CsvReporter> {
+public class CsvStormReporter extends ScheduledStormReporter<CsvReporter> {
     private final static Logger LOG = LoggerFactory.getLogger(CsvStormReporter.class);
 
     public static final String CSV_LOG_DIR = "csv.log.dir";
@@ -55,7 +56,11 @@ public class CsvStormReporter extends SheduledStormReporter<CsvReporter> {
             builder.convertDurationsTo(durationUnit);
         }
 
-        //TODO: expose some simple MetricFilters 
+        StormMetricsFilter filter = getMetricsFilter(reporterConf);
+        if(filter != null){
+            builder.filter(filter);
+        }
+
 
         //defaults to 10
         reportingPeriod = getReportPeriod(reporterConf);

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
index d8d0269..e7dc5f4 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
@@ -21,6 +21,7 @@ import com.codahale.metrics.ganglia.GangliaReporter;
 import com.codahale.metrics.MetricRegistry;
 import info.ganglia.gmetric4j.gmetric.GMetric;
 import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.metrics2.filters.StormMetricsFilter;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -29,7 +30,7 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class GangliaStormReporter extends SheduledStormReporter<GangliaReporter> {
+public class GangliaStormReporter extends ScheduledStormReporter<GangliaReporter> {
     private final static Logger LOG = LoggerFactory.getLogger(GangliaStormReporter.class);
 
     public static final String GANGLIA_HOST = "ganglia.host";
@@ -58,7 +59,10 @@ public class GangliaStormReporter extends SheduledStormReporter<GangliaReporter>
             builder.convertRatesTo(rateUnit);
         }
 
-        //TODO: expose some simple MetricFilters 
+        StormMetricsFilter filter = getMetricsFilter(reporterConf);
+        if(filter != null){
+            builder.filter(filter);
+        }
         String prefix = getMetricsPrefixedWith(reporterConf);
         if (prefix != null) {
             builder.prefixedWith(prefix);

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
index 7a2b31b..0f88fc4 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
@@ -23,6 +23,7 @@ import com.codahale.metrics.graphite.GraphiteUDP;
 import com.codahale.metrics.graphite.Graphite;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.metrics2.filters.StormMetricsFilter;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -30,7 +31,7 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class GraphiteStormReporter extends SheduledStormReporter<GraphiteReporter> {
+public class GraphiteStormReporter extends ScheduledStormReporter<GraphiteReporter> {
     private final static Logger LOG = LoggerFactory.getLogger(GraphiteStormReporter.class);
 
     public static final String GRAPHITE_PREFIXED_WITH = "graphite.prefixed.with";
@@ -53,7 +54,10 @@ public class GraphiteStormReporter extends SheduledStormReporter<GraphiteReporte
             builder.convertRatesTo(rateUnit);
         }
 
-        //TODO: expose some simple MetricFilters 
+        StormMetricsFilter filter = getMetricsFilter(reporterConf);
+        if(filter != null){
+            builder.filter(filter);
+        }
         String prefix = getMetricsPrefixedWith(reporterConf);
         if (prefix != null) {
             builder.prefixedWith(prefix);

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
index 7ac6cde..5b932ea 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
@@ -20,6 +20,7 @@ package org.apache.storm.metrics2.reporters;
 import com.codahale.metrics.JmxReporter;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.storm.daemon.metrics.MetricsUtils;
+import org.apache.storm.metrics2.filters.StormMetricsFilter;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -52,7 +53,10 @@ public class JmxStormReporter implements StormReporter<JmxReporter> {
             builder.inDomain(domain);
         }
 
-        // TODO: expose some simple MetricFilters
+        StormMetricsFilter filter = ScheduledStormReporter.getMetricsFilter(reporterConf);
+        if(filter != null){
+            builder.filter(filter);
+        }
         // other builder functions not exposed:
         //  * createsObjectNamesWith(ObjectNameFactory onFactory) 
         //  * registerWith (MBeanServer)

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
new file mode 100644
index 0000000..940cb19
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.metrics2.reporters;
+
+import com.codahale.metrics.ScheduledReporter;
+import org.apache.storm.metrics2.Metrics2Utils;
+import org.apache.storm.metrics2.filters.StormMetricsFilter;
+import org.apache.storm.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public abstract class ScheduledStormReporter<T extends ScheduledReporter> implements StormReporter{
+    private static final Logger LOG = LoggerFactory.getLogger(ScheduledStormReporter.class);
+    protected ScheduledReporter reporter;
+    long reportingPeriod;
+    TimeUnit reportingPeriodUnit;
+
+    @Override
+    public void start() {
+        if (reporter != null) {
+            LOG.debug("Starting...");
+            reporter.start(reportingPeriod, reportingPeriodUnit);
+        } else {
+            throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName());
+        }
+    }
+
+    @Override
+    public void stop() {
+        if (reporter != null) {
+            LOG.debug("Stopping...");
+            reporter.stop();
+        } else {
+            throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName());
+        }
+    }
+
+
+    static TimeUnit getReportPeriodUnit(Map<String, Object> reporterConf) {
+        TimeUnit unit = getTimeUnitForConfig(reporterConf, REPORT_PERIOD_UNITS);
+        return unit == null ? TimeUnit.SECONDS : unit;
+    }
+
+    private static TimeUnit getTimeUnitForConfig(Map reporterConf, String configName) {
+        String rateUnitString = Utils.getString(reporterConf.get(configName), null);
+        if (rateUnitString != null) {
+            return TimeUnit.valueOf(rateUnitString);
+        }
+        return null;
+    }
+
+    static long getReportPeriod(Map reporterConf) {
+        return Utils.getInt(reporterConf.get(REPORT_PERIOD), 10).longValue();
+    }
+
+    static StormMetricsFilter getMetricsFilter(Map reporterConf){
+        StormMetricsFilter filter = null;
+        Map<String, Object> filterConf = (Map)reporterConf.get("filter");
+        String clazz = (String) filterConf.get("class");
+        if(filterConf != null && clazz != null){
+            try {
+                filter = (StormMetricsFilter) Metrics2Utils.instantiate(clazz);
+                filter.prepare(filterConf);
+            } catch (Exception e) {
+                LOG.warn("Unable to instantiate StormMetricsFilter class: {}", clazz);
+            }
+        }
+        return filter;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/b8de0f36/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java
deleted file mode 100644
index 1b1e7a0..0000000
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/SheduledStormReporter.java
+++ /dev/null
@@ -1,71 +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 org.apache.storm.metrics2.reporters;
-
-import com.codahale.metrics.ScheduledReporter;
-import org.apache.storm.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-public abstract class SheduledStormReporter<T extends ScheduledReporter> implements StormReporter{
-    private static final Logger LOG = LoggerFactory.getLogger(SheduledStormReporter.class);
-    protected ScheduledReporter reporter;
-    long reportingPeriod;
-    TimeUnit reportingPeriodUnit;
-
-    @Override
-    public void start() {
-        if (reporter != null) {
-            LOG.debug("Starting...");
-            reporter.start(reportingPeriod, reportingPeriodUnit);
-        } else {
-            throw new IllegalStateException("Attempt to start without preparing " + getClass().getSimpleName());
-        }
-    }
-
-    @Override
-    public void stop() {
-        if (reporter != null) {
-            LOG.debug("Stopping...");
-            reporter.stop();
-        } else {
-            throw new IllegalStateException("Attempt to stop without preparing " + getClass().getSimpleName());
-        }
-    }
-
-
-    static TimeUnit getReportPeriodUnit(Map<String, Object> reporterConf) {
-        TimeUnit unit = getTimeUnitForConfig(reporterConf, REPORT_PERIOD_UNITS);
-        return unit == null ? TimeUnit.SECONDS : unit;
-    }
-
-    private static TimeUnit getTimeUnitForConfig(Map reporterConf, String configName) {
-        String rateUnitString = Utils.getString(reporterConf.get(configName), null);
-        if (rateUnitString != null) {
-            return TimeUnit.valueOf(rateUnitString);
-        }
-        return null;
-    }
-
-    static long getReportPeriod(Map reporterConf) {
-        return Utils.getInt(reporterConf.get(REPORT_PERIOD), 10).longValue();
-    }
-}


[33/38] storm git commit: STORM-2153: Deprecate old user defined metrics in favor of metrics v2

Posted by pt...@apache.org.
STORM-2153: Deprecate old user defined metrics in favor of metrics v2


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

Branch: refs/heads/1.x-branch
Commit: 8e2f7e7ef621ff20d485bc0af9e6d18803777564
Parents: 6cb81fd
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Jan 11 16:15:04 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Jan 11 16:15:04 2018 -0500

----------------------------------------------------------------------
 storm-core/src/jvm/org/apache/storm/task/TopologyContext.java | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/8e2f7e7e/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
index 330fee1..2068439 100644
--- a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
+++ b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
@@ -313,6 +313,7 @@ public class TopologyContext extends WorkerTopologyContext implements IMetricsCo
      * You must call this during `IBolt.prepare()` or `ISpout.open()`.
      * @return The IMetric argument unchanged.
      */
+    @Deprecated
     public <T extends IMetric> T registerMetric(String name, T metric, int timeBucketSizeInSecs) {
         if((Boolean) _openOrPrepareWasCalled.deref()) {
             throw new RuntimeException("TopologyContext.registerMetric can only be called from within overridden " +
@@ -359,6 +360,7 @@ public class TopologyContext extends WorkerTopologyContext implements IMetricsCo
      *         cause the same metric name can register twice.
      *         So we just return the first metric we meet.
      */
+    @Deprecated
     public IMetric getRegisteredMetricByName(String name) {
         IMetric metric = null;
 
@@ -379,12 +381,14 @@ public class TopologyContext extends WorkerTopologyContext implements IMetricsCo
     /*
      * Convenience method for registering ReducedMetric.
      */
+    @Deprecated
     public ReducedMetric registerMetric(String name, IReducer reducer, int timeBucketSizeInSecs) {
         return registerMetric(name, new ReducedMetric(reducer), timeBucketSizeInSecs);
     }
     /*
      * Convenience method for registering CombinedMetric.
      */
+    @Deprecated
     public CombinedMetric registerMetric(String name, ICombiner combiner, int timeBucketSizeInSecs) {
         return registerMetric(name, new CombinedMetric(combiner), timeBucketSizeInSecs);
     }


[06/38] storm git commit: Merge branch '1.x-branch' into metrics_v2

Posted by pt...@apache.org.
Merge branch '1.x-branch' into metrics_v2


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

Branch: refs/heads/1.x-branch
Commit: 57a50f36df058937587cc08d81c235efc8dc720a
Parents: b8de0f3 c4a09d3
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Aug 30 17:07:02 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Aug 30 17:07:02 2017 -0400

----------------------------------------------------------------------
 .travis.yml                                     |    5 +-
 CHANGELOG.md                                    | 1480 ------------------
 DEVELOPER.md                                    |   11 +-
 bin/storm.cmd                                   |    7 +
 bin/storm.ps1                                   |   68 +
 bin/storm.py                                    |    9 +-
 conf/storm-env.ps1                              |   23 +
 dev-tools/jira-github-join.py                   |    2 +-
 dev-tools/jira/__init__.py                      |  285 ----
 dev-tools/jira_github/__init__.py               |  285 ++++
 dev-tools/release_notes.py                      |  118 ++
 dev-tools/report/report.py                      |    2 +-
 docs/Metrics.md                                 |  201 ++-
 docs/SECURITY.md                                |   40 +-
 docs/cgroups_in_storm.md                        |   71 -
 docs/distcache-blobstore.md                     |    8 +-
 docs/index.md                                   |    1 -
 docs/storm-kafka-client.md                      |  147 +-
 examples/storm-elasticsearch-examples/pom.xml   |   10 +
 .../elasticsearch/bolt/EsIndexTopology.java     |    6 +-
 .../storm/elasticsearch/common/EsTestUtil.java  |   14 +-
 .../trident/TridentEsTopology.java              |    3 +-
 .../TridentKafkaClientWordCountNamedTopics.java |   27 +-
 ...identKafkaClientWordCountWildcardTopics.java |    5 +-
 .../java/org/apache/storm/flux/TCKTest.java     |    9 +
 .../resources/configs/substitution-test.yaml    |    3 +-
 .../src/test/resources/configs/test.properties  |    1 +
 .../backends/trident/TestPlanCompiler.java      |    2 +-
 .../apache/storm/common/AbstractAutoCreds.java  |   27 +-
 .../apache/storm/hbase/security/AutoHBase.java  |   39 +-
 .../apache/storm/hdfs/security/AutoHDFS.java    |   41 +-
 .../apache/storm/hive/security/AutoHive.java    |   39 +-
 external/storm-elasticsearch/pom.xml            |   44 +-
 .../DefaultEsLookupResultOutput.java            |   62 +
 .../elasticsearch/ElasticsearchGetRequest.java  |   36 -
 .../elasticsearch/EsLookupResultOutput.java     |    5 +-
 .../elasticsearch/bolt/AbstractEsBolt.java      |   53 +-
 .../storm/elasticsearch/bolt/EsIndexBolt.java   |   31 +-
 .../storm/elasticsearch/bolt/EsLookupBolt.java  |   49 +-
 .../elasticsearch/bolt/EsPercolateBolt.java     |   41 +-
 .../common/DefaultEsTupleMapper.java            |   19 +
 .../storm/elasticsearch/common/EsConfig.java    |  130 +-
 .../elasticsearch/common/EsTupleMapper.java     |    9 +
 .../common/StormElasticSearchClient.java        |   37 +-
 .../common/TransportAddresses.java              |   72 -
 .../apache/storm/elasticsearch/doc/Index.java   |   69 +
 .../storm/elasticsearch/doc/IndexDoc.java       |   43 +
 .../storm/elasticsearch/doc/IndexItem.java      |   91 ++
 .../storm/elasticsearch/doc/IndexItemDoc.java   |   42 +
 .../apache/storm/elasticsearch/doc/Shards.java  |   63 +
 .../storm/elasticsearch/doc/SourceDoc.java      |   43 +
 .../response/BulkIndexResponse.java             |   80 +
 .../elasticsearch/response/LookupResponse.java  |   63 +
 .../response/PercolateResponse.java             |   85 +
 .../storm/elasticsearch/trident/EsState.java    |   72 +-
 .../elasticsearch/trident/EsStateFactory.java   |   15 +-
 .../bolt/AbstractEsBoltIntegrationTest.java     |   68 +-
 .../elasticsearch/bolt/AbstractEsBoltTest.java  |   15 +-
 .../elasticsearch/bolt/EsIndexBoltTest.java     |   44 +-
 .../bolt/EsLookupBoltIntegrationTest.java       |   75 +-
 .../elasticsearch/bolt/EsLookupBoltTest.java    |   67 +-
 .../elasticsearch/bolt/EsPercolateBoltTest.java |   62 +-
 .../elasticsearch/common/EsConfigTest.java      |   60 +-
 .../storm/elasticsearch/common/EsTestUtil.java  |  101 +-
 .../common/TransportAddressesTest.java          |   81 -
 .../trident/EsStateFactoryTest.java             |    2 +-
 .../elasticsearch/trident/EsStateTest.java      |   98 ++
 .../src/test/resources/log4j2.xml               |   33 +
 .../storm/hdfs/bolt/AbstractHdfsBolt.java       |   23 +-
 .../java/org/apache/storm/hdfs/bolt/Writer.java |   35 +
 .../storm/hdfs/common/AbstractHDFSWriter.java   |   16 +-
 .../org/apache/storm/jms/spout/JmsSpout.java    |  291 +++-
 .../apache/storm/jms/spout/JmsSpoutTest.java    |   81 +-
 .../kafka/spout/EmptyKafkaTupleListener.java    |   53 +
 .../apache/storm/kafka/spout/KafkaSpout.java    |   15 +-
 .../storm/kafka/spout/KafkaSpoutConfig.java     |  574 ++++---
 .../storm/kafka/spout/KafkaTupleListener.java   |   83 +
 .../spout/ManualPartitionNamedSubscription.java |   78 -
 .../ManualPartitionPatternSubscription.java     |   76 -
 .../spout/ManualPartitionSubscription.java      |   72 +
 .../storm/kafka/spout/ManualPartitioner.java    |    4 +-
 .../storm/kafka/spout/NamedSubscription.java    |    4 +-
 .../storm/kafka/spout/NamedTopicFilter.java     |   68 +
 .../storm/kafka/spout/PatternSubscription.java  |    4 +-
 .../storm/kafka/spout/PatternTopicFilter.java   |   70 +
 .../kafka/spout/SerializableDeserializer.java   |    6 +-
 .../apache/storm/kafka/spout/Subscription.java  |    2 +-
 .../apache/storm/kafka/spout/TopicFilter.java   |   38 +
 .../internal/KafkaConsumerFactoryDefault.java   |    3 +-
 .../kafka/spout/internal/OffsetManager.java     |    9 +-
 .../spout/trident/KafkaTridentSpoutManager.java |    3 +-
 .../storm/kafka/spout/KafkaSpoutCommitTest.java |  131 ++
 .../storm/kafka/spout/KafkaSpoutConfigTest.java |   17 +-
 .../storm/kafka/spout/KafkaSpoutEmitTest.java   |   53 +-
 .../kafka/spout/KafkaSpoutRebalanceTest.java    |   37 +-
 .../kafka/spout/KafkaSpoutRetryLimitTest.java   |  116 ++
 .../kafka/spout/MaxUncommittedOffsetTest.java   |   10 +-
 .../storm/kafka/spout/NamedTopicFilterTest.java |   70 +
 .../kafka/spout/PatternTopicFilterTest.java     |   75 +
 .../kafka/spout/SingleTopicKafkaSpoutTest.java  |   10 +-
 .../SpoutWithMockedConsumerSetupHelper.java     |   87 +
 .../SingleTopicKafkaSpoutConfiguration.java     |   48 +-
 .../test/KafkaSpoutTopologyMainNamedTopics.java |    4 +-
 .../KafkaSpoutTopologyMainWildcardTopics.java   |    4 +-
 integration-test/config/install-zookeeper.sh    |    2 +-
 integration-test/run-it.sh                      |    3 +-
 .../org/apache/storm/command/config_value.clj   |    6 +-
 .../src/clj/org/apache/storm/converter.clj      |   11 +-
 .../src/clj/org/apache/storm/daemon/common.clj  |    4 +-
 .../clj/org/apache/storm/daemon/executor.clj    |   14 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |   71 +-
 .../storm/pacemaker/pacemaker_state_factory.clj |   15 +-
 storm-core/src/jvm/org/apache/storm/Config.java |   16 +-
 .../storm/cluster/StormClusterStateImpl.java    |   16 +-
 .../storm/daemon/supervisor/AdvancedFSOps.java  |   16 +-
 .../storm/daemon/supervisor/Container.java      |    6 +-
 .../daemon/supervisor/ReadClusterState.java     |    3 +
 .../apache/storm/daemon/supervisor/Slot.java    |   11 +
 .../storm/daemon/supervisor/Supervisor.java     |   16 +-
 .../daemon/supervisor/SupervisorUtils.java      |   11 +-
 .../daemon/supervisor/timer/UpdateBlobs.java    |   26 +-
 .../jvm/org/apache/storm/drpc/DRPCSpout.java    |    6 -
 .../org/apache/storm/generated/Assignment.java  |  114 +-
 .../apache/storm/generated/LocalAssignment.java |  114 +-
 .../org/apache/storm/generated/StormBase.java   |  114 +-
 .../apache/storm/localizer/AsyncLocalizer.java  |   37 +-
 .../jvm/org/apache/storm/scheduler/Cluster.java |   85 +-
 .../apache/storm/scheduler/TopologyDetails.java |   35 +-
 .../multitenant/MultitenantScheduler.java       |    2 +-
 .../storm/security/INimbusCredentialPlugin.java |   24 +-
 .../security/auth/ICredentialsRenewer.java      |   18 +-
 .../storm/security/auth/kerberos/AutoTGT.java   |    6 +-
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   31 +-
 .../org/apache/storm/utils/DisruptorQueue.java  |   70 +-
 .../org/apache/storm/utils/ObjectReader.java    |   58 +
 .../src/jvm/org/apache/storm/utils/Utils.java   |   52 +-
 storm-core/src/py/storm/ttypes.py               |   47 +-
 storm-core/src/storm.thrift                     |    5 +
 .../test/clj/org/apache/storm/cluster_test.clj  |    8 +-
 .../scheduler/multitenant_scheduler_test.clj    |   77 +-
 .../scheduler/resource_aware_scheduler_test.clj |   49 +-
 .../clj/org/apache/storm/scheduler_test.clj     |   10 +-
 .../test/jvm/org/apache/storm/ConfigTest.java   |   92 ++
 .../storm/daemon/supervisor/ContainerTest.java  |    6 +-
 .../storm/localizer/AsyncLocalizerTest.java     |    8 +-
 .../org/apache/storm/scheduler/ClusterTest.java |  111 ++
 .../resource/TestResourceAwareScheduler.java    |  183 ++-
 .../storm/scheduler/resource/TestUser.java      |    7 +-
 .../TestUtilsForResourceAwareScheduler.java     |   34 +-
 .../eviction/TestDefaultEvictionStrategy.java   |  118 +-
 .../TestDefaultResourceAwareStrategy.java       |    8 +-
 .../org/apache/storm/utils/ConfigUtilsTest.java |   98 ++
 .../jvm/org/apache/storm/utils/UtilsTest.java   |   84 +
 storm-dist/binary/src/main/assembly/binary.xml  |   10 +-
 .../src/main/resources/resources/storm.py       |    2 +-
 155 files changed, 5523 insertions(+), 3666 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/57a50f36/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 95e43f6,52063fc..250ace1
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -233,10 -231,7 +233,10 @@@
                                    (str "executor"  executor-id "-send-queue")
                                    (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
                                    (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
 +                                  (.getStormId worker-context)
 +                                  component-id
 +                                  (.getThisWorkerPort worker-context)
-                                   :producer-type :single-threaded
+                                   :producer-type :multi-threaded
                                    :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                    :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
          ]
@@@ -820,9 -811,8 +822,9 @@@
                           (let [delta (tuple-time-delta! tuple)]
                             (when debug? 
                               (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
 +                           (.mark  ^Meter (:acked-meter (:executor-data task-data)))
                             (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
-                            (when delta
+                            (when (<= 0 delta)
                               (stats/bolt-acked-tuple! executor-stats
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)
@@@ -836,9 -826,8 +838,9 @@@
                                 debug? (= true (storm-conf TOPOLOGY-DEBUG))]
                             (when debug? 
                               (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
 +                           (.mark  ^Meter (:failed-meter (:executor-data task-data)))
                             (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
-                            (when delta
+                            (when (<= 0 delta)
                               (stats/bolt-failed-tuple! executor-stats
                                                         (.getSourceComponent tuple)
                                                         (.getSourceStreamId tuple)

http://git-wip-us.apache.org/repos/asf/storm/blob/57a50f36/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/57a50f36/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
index 5c0a2fb,5fd4b84..ca8568c
--- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
@@@ -34,9 -34,6 +34,8 @@@ import com.lmax.disruptor.dsl.ProducerT
  import org.apache.storm.Config;
  import org.apache.storm.metric.api.IStatefulObject;
  import org.apache.storm.metric.internal.RateTracker;
 +import org.apache.storm.metrics2.DisruptorMetrics;
 +import org.apache.storm.metrics2.StormMetricRegistry;
- import org.apache.storm.task.WorkerTopologyContext;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -62,19 -59,18 +61,19 @@@ import java.util.concurrent.locks.Reent
   * the ability to catch up to the producer by processing tuples in batches.
   */
  public class DisruptorQueue implements IStatefulObject {
--    private static final Logger LOG = LoggerFactory.getLogger(DisruptorQueue.class);    
++    private static final Logger LOG = LoggerFactory.getLogger(DisruptorQueue.class);
      private static final Object INTERRUPT = new Object();
      private static final String PREFIX = "disruptor-";
      private static final FlusherPool FLUSHER = new FlusherPool();
--    
 +    private static final Timer METRICS_TIMER = new Timer("disruptor-metrics-timer", true);
++
      private static int getNumFlusherPoolThreads() {
          int numThreads = 100;
          try {
--        	Map<String, Object> conf = Utils.readStormConfig();
--        	numThreads = Utils.getInt(conf.get(Config.STORM_WORKER_DISRUPTOR_FLUSHER_MAX_POOL_SIZE), numThreads);
++            Map<String, Object> conf = Utils.readStormConfig();
++            numThreads = Utils.getInt(conf.get(Config.STORM_WORKER_DISRUPTOR_FLUSHER_MAX_POOL_SIZE), numThreads);
          } catch (Exception e) {
--        	LOG.warn("Error while trying to read system config", e);
++            LOG.warn("Error while trying to read system config", e);
          }
          try {
              String threads = System.getProperty("num_flusher_pool_threads", String.valueOf(numThreads));
@@@ -86,8 -82,8 +85,8 @@@
          return numThreads;
      }
  
--    private static class FlusherPool { 
--    	private static final String THREAD_PREFIX = "disruptor-flush";
++    private static class FlusherPool {
++        private static final String THREAD_PREFIX = "disruptor-flush";
          private Timer _timer = new Timer(THREAD_PREFIX + "-trigger", true);
          private ThreadPoolExecutor _exec;
          private HashMap<Long, ArrayList<Flusher>> _pendingFlush = new HashMap<>();
@@@ -201,8 -197,8 +200,8 @@@
              if (block) {
                  _flushLock.lock();
              } else if (!_flushLock.tryLock()) {
--               //Someone else if flushing so don't do anything
--               return;
++                //Someone else if flushing so don't do anything
++                return;
              }
              try {
                  while (!_overflow.isEmpty()) {
@@@ -256,7 -252,7 +255,7 @@@
                      }
                  }
  
--                if (!flushed) {        
++                if (!flushed) {
                      _overflow.add(_currentBatch);
                      _currentBatch = new ArrayList<Object>(_inputBatchSize);
                  }
@@@ -276,8 -272,8 +275,8 @@@
              if (block) {
                  _flushLock.lock();
              } else if (!_flushLock.tryLock()) {
--               //Someone else if flushing so don't do anything
--               return;
++                //Someone else if flushing so don't do anything
++                return;
              }
              try {
                  while (!_overflow.isEmpty()) {
@@@ -349,11 -345,9 +348,17 @@@
              return (1.0F * population() / capacity());
          }
  
 +        public double arrivalRate(){
 +            return _rateTracker.reportRate();
 +        }
 +
 +        public double sojournTime(){
++            return tuplePopulation.get() / Math.max(arrivalRate(), 0.00001) * 1000.0;
++        }
++
+         public Object getState() {
+             Map state = new HashMap<String, Object>();
+ 
              // get readPos then writePos so it's never an under-estimate
              long rp = readPos();
              long wp = writePos();
@@@ -393,8 -393,7 +404,8 @@@
      private final int _inputBatchSize;
      private final ConcurrentHashMap<Long, ThreadLocalInserter> _batchers = new ConcurrentHashMap<Long, ThreadLocalInserter>();
      private final Flusher _flusher;
-     private final QueueMetrics _metrics; // old metrics API
+     private final QueueMetrics _metrics;
 +    private final DisruptorMetrics _disruptorMetrics;
  
      private String _queueName = "";
      private DisruptorBackpressureCallback _cb = null;
@@@ -402,9 -401,10 +413,10 @@@
      private int _lowWaterMark = 0;
      private boolean _enableBackpressure = false;
      private final AtomicLong _overflowCount = new AtomicLong(0);
+     private final AtomicLong tuplePopulation = new AtomicLong(0);
      private volatile boolean _throttleOn = false;
  
 -    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval) {
 +    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval, String topologyId, String componentId, int port) {
          this._queueName = PREFIX + queueName;
          WaitStrategy wait;
          if (readTimeout <= 0) {
@@@ -425,13 -424,6 +437,12 @@@
  
          _flusher = new Flusher(Math.max(flushInterval, 1), _queueName);
          _flusher.start();
- 
 +        METRICS_TIMER.schedule(new TimerTask(){
 +            @Override
 +            public void run() {
 +                _disruptorMetrics.set(_metrics);
 +            }
-         }, 15000, 15000); // TODO: Configurable interval
++        }, 15000, 15000);
      }
  
      public String getName() {
@@@ -619,4 -612,4 +631,4 @@@
      public QueueMetrics getMetrics() {
          return _metrics;
      }
--}
++}


[20/38] storm git commit: STORM-2153: Add warnings for component names containing '.' to DefaultTopologyValidator; Add StrictTopologyValidator

Posted by pt...@apache.org.
STORM-2153: Add warnings for component names containing '.' to DefaultTopologyValidator; Add StrictTopologyValidator


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

Branch: refs/heads/1.x-branch
Commit: 5ce45b72715da4478fddabdfde12becec1373a93
Parents: cd272c4
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Dec 14 14:06:20 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Dec 14 14:06:20 2017 -0500

----------------------------------------------------------------------
 .../storm/nimbus/DefaultTopologyValidator.java  | 36 ++++++++++-
 .../storm/nimbus/StrictTopologyValidator.java   | 65 ++++++++++++++++++++
 2 files changed, 100 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5ce45b72/storm-core/src/jvm/org/apache/storm/nimbus/DefaultTopologyValidator.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/nimbus/DefaultTopologyValidator.java b/storm-core/src/jvm/org/apache/storm/nimbus/DefaultTopologyValidator.java
index 0626cb6..fc0dfac 100644
--- a/storm-core/src/jvm/org/apache/storm/nimbus/DefaultTopologyValidator.java
+++ b/storm-core/src/jvm/org/apache/storm/nimbus/DefaultTopologyValidator.java
@@ -17,15 +17,49 @@
  */
 package org.apache.storm.nimbus;
 
+import org.apache.storm.generated.Bolt;
 import org.apache.storm.generated.InvalidTopologyException;
+import org.apache.storm.generated.SpoutSpec;
 import org.apache.storm.generated.StormTopology;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.util.Map;
 
 public class DefaultTopologyValidator implements ITopologyValidator {
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultTopologyValidator.class);
     @Override
     public void prepare(Map StormConf){
     }
     @Override
-    public void validate(String topologyName, Map topologyConf, StormTopology topology) throws InvalidTopologyException {        
+    public void validate(String topologyName, Map topologyConf, StormTopology topology) throws InvalidTopologyException {
+        if(topologyName.contains(".")){
+            LOG.warn("Metrics for topology name '{}' will be reported as '{}'.", topologyName, topologyName.replace('.', '_') );
+        }
+        Map<String, SpoutSpec> spouts = topology.get_spouts();
+        for(String spoutName : spouts.keySet()){
+            if(spoutName.contains(".")){
+                LOG.warn("Metrics for spout name '{}' will be reported as '{}'.", spoutName, spoutName.replace('.', '_') );
+            }
+            SpoutSpec spoutSpec = spouts.get(spoutName);
+            for(String streamName : spoutSpec.get_common().get_streams().keySet()){
+                if(streamName.contains(".")){
+                    LOG.warn("Metrics for stream name '{}' will be reported as '{}'.", streamName, streamName.replace('.', '_') );
+                }
+            }
+        }
+
+        Map<String, Bolt> bolts = topology.get_bolts();
+        for(String boltName : bolts.keySet()){
+            if(boltName.contains(".")){
+                LOG.warn("Metrics for bolt name '{}' will be reported as '{}'.", boltName, boltName.replace('.', '_') );
+            }
+            Bolt bolt = bolts.get(boltName);
+            for(String streamName : bolt.get_common().get_streams().keySet()){
+                if(streamName.contains(".")){
+                    LOG.warn("Metrics for stream name '{}' will be reported as '{}'.", streamName, streamName.replace('.', '_') );
+                }
+            }
+        }
     }    
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/5ce45b72/storm-core/src/jvm/org/apache/storm/nimbus/StrictTopologyValidator.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/nimbus/StrictTopologyValidator.java b/storm-core/src/jvm/org/apache/storm/nimbus/StrictTopologyValidator.java
new file mode 100644
index 0000000..e081a4d
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/nimbus/StrictTopologyValidator.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.storm.nimbus;
+
+import org.apache.storm.generated.Bolt;
+import org.apache.storm.generated.InvalidTopologyException;
+import org.apache.storm.generated.SpoutSpec;
+import org.apache.storm.generated.StormTopology;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+
+public class StrictTopologyValidator implements ITopologyValidator {
+    private static final Logger LOG = LoggerFactory.getLogger(StrictTopologyValidator.class);
+    @Override
+    public void prepare(Map StormConf){
+    }
+    @Override
+    public void validate(String topologyName, Map topologyConf, StormTopology topology) throws InvalidTopologyException {
+        if(topologyName.contains(".")){
+            throw new InvalidTopologyException(String.format("Topology name '%s' contains illegal character '.'", topologyName));
+        }
+        Map<String, SpoutSpec> spouts = topology.get_spouts();
+        for(String spoutName : spouts.keySet()){
+            if(spoutName.contains(".")){
+                throw new InvalidTopologyException(String.format("Spout name '%s' contains illegal character '.'", spoutName));
+            }
+            SpoutSpec spoutSpec = spouts.get(spoutName);
+            for(String streamName : spoutSpec.get_common().get_streams().keySet()){
+                if(streamName.contains(".")){
+                    throw new InvalidTopologyException(String.format("Stream name '%s' contains illegal character '.'", streamName));
+                }
+            }
+        }
+
+        Map<String, Bolt> bolts = topology.get_bolts();
+        for(String boltName : bolts.keySet()){
+            if(boltName.contains(".")){
+                throw new InvalidTopologyException(String.format("Bolt name '%s' contains illegal character '.'", boltName));
+            }
+            Bolt bolt = bolts.get(boltName);
+            for(String streamName : bolt.get_common().get_streams().keySet()){
+                if(streamName.contains(".")){
+                    throw new InvalidTopologyException(String.format("Stream name '%s' contains illegal character '.'", streamName));
+                }
+            }
+        }
+    }    
+}


[23/38] storm git commit: STORM-2153: address review comments

Posted by pt...@apache.org.
STORM-2153: address review comments


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

Branch: refs/heads/1.x-branch
Commit: 868de5b33b8145d787a9b3d08bdac6908591790d
Parents: b5ae9c3
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Dec 22 14:42:35 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Dec 22 14:42:35 2017 -0500

----------------------------------------------------------------------
 .../reporters/ScheduledStormReporter.java       |  2 +-
 .../org/apache/storm/utils/DisruptorQueue.java  | 18 ++++++++-----
 .../storm/validation/ConfigValidation.java      | 28 ++++++++++++--------
 3 files changed, 30 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/868de5b3/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
index e88b41b..b7ffa61 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
@@ -81,7 +81,7 @@ public abstract class ScheduledStormReporter implements StormReporter{
                     filter = (StormMetricsFilter) Metrics2Utils.instantiate(clazz);
                     filter.prepare(filterConf);
                 } catch (Exception e) {
-                    LOG.warn("Unable to instantiate StormMetricsFilter class: {}", clazz);
+                    throw new RuntimeException("Unable to instantiate StormMetricsFilter class: " + clazz);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/868de5b3/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
index ca8568c..d7cf401 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
@@ -437,12 +437,17 @@ public class DisruptorQueue implements IStatefulObject {
 
         _flusher = new Flusher(Math.max(flushInterval, 1), _queueName);
         _flusher.start();
-        METRICS_TIMER.schedule(new TimerTask(){
-            @Override
-            public void run() {
-                _disruptorMetrics.set(_metrics);
-            }
-        }, 15000, 15000);
+        try {
+            METRICS_TIMER.schedule(new TimerTask() {
+                @Override
+                public void run() {
+                    _disruptorMetrics.set(_metrics);
+                }
+            }, 15000, 15000);
+        } catch (IllegalStateException e){
+            // Ignore. IllegalStateException is thrown by Timer.schedule() if the timer
+            // has been cancelled. (This happens in unit tests)
+        }
     }
 
     public String getName() {
@@ -458,6 +463,7 @@ public class DisruptorQueue implements IStatefulObject {
             publishDirect(new ArrayList<Object>(Arrays.asList(INTERRUPT)), true);
             _flusher.close();
             _metrics.close();
+            METRICS_TIMER.cancel();
         } catch (InsufficientCapacityException e) {
             //This should be impossible
             throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/storm/blob/868de5b3/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
index d7ca48d..9d9db33 100644
--- a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
@@ -494,6 +494,12 @@ public class ConfigValidation {
     }
 
     public static class MetricReportersValidator extends Validator {
+        private static final String NIMBUS = "nimbus";
+        private static final String SUPERVISOR = "supervisor";
+        private static final String WORKER = "worker";
+        private static final String CLASS = "class";
+        private static final String FILTER = "filter";
+        private static final String DAEMONS = "daemons";
 
         @Override
         public void validateField(String name, Object o) {
@@ -501,23 +507,23 @@ public class ConfigValidation {
                 return;
             }
             SimpleTypeValidator.validateField(name, Map.class, o);
-            if(!((Map) o).containsKey("class") ) {
+            if(!((Map) o).containsKey(CLASS) ) {
                 throw new IllegalArgumentException( "Field " + name + " must have map entry with key: class");
             }
-            if(!((Map) o).containsKey("daemons") ) {
+            if(!((Map) o).containsKey(DAEMONS) ) {
                 throw new IllegalArgumentException("Field " + name + " must have map entry with key: daemons");
             } else {
                 // daemons can only be 'nimbus', 'supervisor', or 'worker'
-                Object list = ((Map)o).get("daemons");
-                if(list == null || !(list instanceof List)){
+                Object list = ((Map)o).get(DAEMONS);
+                if(!(list instanceof List)){
                     throw new IllegalArgumentException("Field 'daemons' must be a non-null list.");
                 }
                 List daemonList = (List)list;
                 for(Object string : daemonList){
                     if (string instanceof String &&
-                            (((String) string).equals("nimbus") ||
-                                    ((String) string).equals("supervisor") ||
-                                    ((String) string).equals("worker"))) {
+                            (string.equals(NIMBUS) ||
+                                    string.equals(SUPERVISOR) ||
+                                    string.equals(WORKER))) {
                         continue;
                     }
                     throw new IllegalArgumentException("Field 'daemons' must contain at least one of the following:" +
@@ -525,11 +531,11 @@ public class ConfigValidation {
                 }
 
             }
-            if(((Map)o).containsKey("filter")){
-                Map filterMap = (Map)((Map)o).get("filter");
-                SimpleTypeValidator.validateField("class", String.class, filterMap.get("class"));
+            if(((Map)o).containsKey(FILTER)){
+                Map filterMap = (Map)((Map)o).get(FILTER);
+                SimpleTypeValidator.validateField(CLASS, String.class, filterMap.get(CLASS));
             }
-            SimpleTypeValidator.validateField(name, String.class, ((Map) o).get("class"));
+            SimpleTypeValidator.validateField(name, String.class, ((Map) o).get(CLASS));
 
         }
     }


[16/38] storm git commit: address review comments

Posted by pt...@apache.org.
address review comments


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

Branch: refs/heads/1.x-branch
Commit: a9647d8bffd2df8c7e3dcc31d8ce720babf4d0f8
Parents: 85dbacd
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Dec 8 15:14:53 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Dec 8 15:14:53 2017 -0500

----------------------------------------------------------------------
 conf/defaults.yaml                                  |  1 -
 .../org/apache/storm/metrics2/DisruptorMetrics.java | 16 ++++++++--------
 .../apache/storm/metrics2/StormMetricRegistry.java  |  2 +-
 .../storm/metrics2/filters/StormMetricsFilter.java  |  2 +-
 .../metrics2/reporters/GangliaStormReporter.java    |  3 ---
 .../apache/storm/validation/ConfigValidation.java   |  7 ++++---
 6 files changed, 14 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a9647d8b/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index e51b50c..435a7e8 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -314,7 +314,6 @@ storm.cluster.metrics.consumer.publish.interval.secs: 60
 #        - "worker"
 #    report.period: 10
 #    report.period.units: "SECONDS"
-#
 #    filter:
 #        class: "org.apache.storm.metrics2.filters.RegexFilter"
 #        expression: ".*my_component.*emitted.*"

http://git-wip-us.apache.org/repos/asf/storm/blob/a9647d8b/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java b/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
index 22eb6c5..42f698c 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
@@ -20,14 +20,14 @@ package org.apache.storm.metrics2;
 import org.apache.storm.utils.DisruptorQueue;
 
 public class DisruptorMetrics {
-    private SimpleGauge<Long> capacity;
-    private SimpleGauge<Long> population;
-    private SimpleGauge<Long> writePosition;
-    private SimpleGauge<Long> readPosition;
-    private SimpleGauge<Double> arrivalRate;
-    private SimpleGauge<Double> sojournTime;
-    private SimpleGauge<Long> overflow;
-    private SimpleGauge<Float> pctFull;
+    private final SimpleGauge<Long> capacity;
+    private final SimpleGauge<Long> population;
+    private final SimpleGauge<Long> writePosition;
+    private final SimpleGauge<Long> readPosition;
+    private final SimpleGauge<Double> arrivalRate;
+    private final SimpleGauge<Double> sojournTime;
+    private final SimpleGauge<Long> overflow;
+    private final SimpleGauge<Float> pctFull;
 
 
     DisruptorMetrics(SimpleGauge<Long> capacity,

http://git-wip-us.apache.org/repos/asf/storm/blob/a9647d8b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 912d888..200ddcf 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -83,7 +83,7 @@ public class StormMetricRegistry {
         try {
             hostName = Utils.localHostname();
         } catch (UnknownHostException e) {
-             LOG.warn("Unable to determine hostname while starting the metrics system. Hostname ill be reported" +
+             LOG.warn("Unable to determine hostname while starting the metrics system. Hostname will be reported" +
                      " as 'localhost'.");
         }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a9647d8b/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java b/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java
index 57f7255..8e6b95f 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/filters/StormMetricsFilter.java
@@ -25,7 +25,7 @@ public interface StormMetricsFilter extends MetricFilter {
 
     /**
      * Called after the filter is instantiated.
-     * @param config an arbitrary configuration map pulled from the yaml configuration.
+     * @param config A map of the properties from the 'filter' section of the reporter configuration.
      */
     void prepare(Map<String, Object> config);
 

http://git-wip-us.apache.org/repos/asf/storm/blob/a9647d8b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
index 09af2e1..497cb04 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
@@ -84,9 +84,6 @@ public class GangliaStormReporter extends ScheduledStormReporter {
         //defaults to seconds
         reportingPeriodUnit = getReportPeriodUnit(reporterConf);
 
-        // Not exposed:
-        // * withClock(Clock)
-
         String group = getMetricsTargetUDPGroup(reporterConf);
         Integer port = getMetricsTargetPort(reporterConf);
         String udpAddressingMode = getMetricsTargetUDPAddressingMode(reporterConf);

http://git-wip-us.apache.org/repos/asf/storm/blob/a9647d8b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
index 8bcea84..89aecee 100644
--- a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
@@ -518,15 +518,16 @@ public class ConfigValidation {
                             (((String) string).equals("nimbus") ||
                                     ((String) string).equals("supervisor") ||
                                     ((String) string).equals("worker"))) {
-                        return;
+                        continue;
                     }
-                    throw new IllegalArgumentException("Field daemons must contain at least one of \"nimbus\", \"supervisor\", or \"worker\"");
+                    throw new IllegalArgumentException("Field 'daemons' must contain at least one of the following:" +
+                            " \"nimbus\", \"supervisor\", or \"worker\"");
                 }
 
             }
             if(((Map)o).containsKey("filter")){
                 Map filterMap = (Map)((Map)o).get("filter");
-                SimpleTypeValidator.validateField("filter", String.class, filterMap.get("class"));
+                SimpleTypeValidator.validateField("class", String.class, filterMap.get("class"));
             }
             SimpleTypeValidator.validateField(name, String.class, ((Map) o).get("class"));
 


[38/38] storm git commit: Merge branch '1.x-branch' into metrics_v2

Posted by pt...@apache.org.
Merge branch '1.x-branch' into metrics_v2


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

Branch: refs/heads/1.x-branch
Commit: c4404cab69966fbf98c4f1e6d4bc41945e51cfcb
Parents: 427076e 25fa9dd
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Mon Jan 22 11:36:12 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Mon Jan 22 11:36:12 2018 -0500

----------------------------------------------------------------------
 .travis.yml                                     |  10 +-
 DEVELOPER.md                                    |  10 +-
 docs/Contributing-to-Storm.md                   |   2 +-
 examples/storm-solr-examples/pom.xml            |  21 +
 examples/storm-starter/README.markdown          |   4 +-
 .../multilang/resources/randomsentence.js       |   8 +
 .../main/java/org/apache/storm/flux/Flux.java   |  12 +-
 .../apache/storm/flux/parser/FluxParser.java    |  73 ++--
 .../java/org/apache/storm/flux/TCKTest.java     |   4 +-
 .../multilang/MultilangEnvironmentTest.java     |  89 ++++
 .../multilang/MultilangEnvirontmentTest.java    |  89 ----
 .../resources/configs/substitution-test.yaml    |   6 +-
 .../main/resources/resources/randomsentence.js  |   8 +
 .../apache/storm/common/AbstractAutoCreds.java  |  16 +-
 external/storm-druid/pom.xml                    |  17 +
 external/storm-hdfs/pom.xml                     |   8 +
 external/storm-hive/README.md                   |   4 +-
 external/storm-hive/pom.xml                     |   8 +
 external/storm-kafka-client/pom.xml             |  16 +
 .../apache/storm/kafka/spout/KafkaSpout.java    | 245 ++++++++---
 .../storm/kafka/spout/KafkaSpoutConfig.java     |  54 ++-
 .../storm/kafka/spout/KafkaSpoutMessageId.java  |  26 +-
 .../spout/ManualPartitionSubscription.java      |   7 +-
 .../kafka/spout/internal/CommitMetadata.java    |  63 +++
 .../kafka/spout/internal/OffsetManager.java     |  89 ++--
 .../kafka/spout/metrics/KafkaOffsetMetric.java  | 141 +++++++
 .../storm/kafka/trident/TridentKafkaState.java  |  70 ++--
 .../kafka/trident/TridentKafkaStateFactory.java |  17 +-
 .../kafka/trident/TridentKafkaStateUpdater.java |   6 +-
 .../kafka/spout/KafkaSpoutAbstractTest.java     | 179 ++++++++
 .../storm/kafka/spout/KafkaSpoutConfigTest.java |  10 +
 .../storm/kafka/spout/KafkaSpoutEmitTest.java   |  10 +-
 .../KafkaSpoutLogCompactionSupportTest.java     |   2 +-
 .../spout/KafkaSpoutMessagingGuaranteeTest.java |  14 +-
 .../kafka/spout/KafkaSpoutReactivationTest.java | 145 +++++++
 .../kafka/spout/KafkaSpoutRebalanceTest.java    |   5 +-
 .../kafka/spout/KafkaSpoutRetryLimitTest.java   |   5 +-
 .../kafka/spout/KafkaSpoutSingleTopicTest.java  | 379 +++++++++++++++++
 ...outTopologyDeployActivateDeactivateTest.java | 116 ++++++
 .../kafka/spout/MaxUncommittedOffsetTest.java   |   7 +-
 .../kafka/spout/SingleTopicKafkaSpoutTest.java  | 410 -------------------
 .../spout/SingleTopicKafkaUnitSetupHelper.java  |  24 +-
 .../SpoutWithMockedConsumerSetupHelper.java     |  66 ++-
 .../kafka/spout/internal/OffsetManagerTest.java |  35 +-
 .../ManualPartitionSubscriptionTest.java        |   2 +
 .../org/apache/storm/kafka/KafkaUtilsTest.java  |   2 +-
 external/storm-mqtt/pom.xml                     |   2 -
 external/storm-solr/README.md                   |  26 +-
 external/storm-solr/pom.xml                     |  27 +-
 .../apache/storm/solr/bolt/SolrUpdateBolt.java  |  28 +-
 .../apache/storm/solr/config/SolrConfig.java    |  14 +
 .../storm/solr/mapper/SolrJsonMapper.java       |   1 +
 .../schema/builder/RestJsonSchemaBuilder.java   |   1 +
 .../apache/storm/solr/trident/SolrState.java    |   4 +
 integration-test/pom.xml                        |   2 +
 pom.xml                                         |   4 +-
 storm-core/pom.xml                              |  79 ++--
 storm-core/src/dev/resources/tester_spout.js    |   8 +
 storm-core/src/jvm/org/apache/storm/Config.java |   8 +
 .../daemon/supervisor/ReadClusterState.java     |   1 +
 .../apache/storm/daemon/supervisor/Slot.java    |  13 +-
 .../storm/daemon/supervisor/Supervisor.java     |   2 +-
 .../storm/metric/FileBasedEventLogger.java      |  34 +-
 .../jvm/org/apache/storm/spout/ShellSpout.java  |  37 +-
 .../jvm/org/apache/storm/task/ShellBolt.java    |  38 +-
 .../storm/utils/DefaultShellLogHandler.java     | 113 +++++
 .../org/apache/storm/utils/ShellLogHandler.java |  52 +++
 .../jvm/org/apache/storm/utils/ShellUtils.java  |  17 +
 .../storm/utils/WorkerBackpressureThread.java   |  20 +-
 .../src/resources/compile-worker-launcher.sh    |  21 +
 .../storm/utils/DefaultShellLogHandlerTest.java | 105 +++++
 .../org/apache/storm/utils/ShellUtilsTest.java  | 103 +++++
 .../utils/WorkerBackpressureThreadTest.java     |  23 +-
 .../src/main/resources/resources/storm.js       |  20 +
 .../src/main/resources/resources/storm.py       |  10 +
 .../ruby/src/main/resources/resources/storm.rb  |   8 +
 76 files changed, 2408 insertions(+), 947 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/storm/blob/c4404cab/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------


[18/38] storm git commit: STORM-2153: add streamId and executorId to metrics names; replace '.' with '_' in metrics names

Posted by pt...@apache.org.
STORM-2153: add streamId and executorId to metrics names; replace '.' with '_' in metrics names


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

Branch: refs/heads/1.x-branch
Commit: c91da676e7fc550ab44fd2d4d91dc95243059eb7
Parents: dd977e8
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Dec 13 15:25:56 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Dec 13 15:25:56 2017 -0500

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    |  6 ++--
 .../src/clj/org/apache/storm/daemon/task.clj    |  7 ++---
 .../storm/metrics2/StormMetricRegistry.java     | 32 +++++++++++++++-----
 3 files changed, 30 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/c91da676/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 0aca4bd..fa7d44c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -280,8 +280,6 @@
                                (log-message "Got interrupted excpetion shutting thread down...")
                                ((:suicide-fn <>))))
      :sampler (mk-stats-sampler storm-conf)
-     :failed-counter (StormMetricRegistry/counter "failed" worker-context component-id)
-     :acked-counter (StormMetricRegistry/counter "acked" worker-context component-id)
      :spout-throttling-metrics (if (= executor-type :spout)
                                 (builtin-metrics/make-spout-throttling-data)
                                 nil)
@@ -444,7 +442,7 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-failed-tuple! (:stats executor-data) (:failed-counter executor-data) (:stream tuple-info) time-delta))))
+      (stats/spout-failed-tuple! (:stats executor-data) (StormMetricRegistry/counter "failed" worker-context (:component-id executor-data) (:executor-id executor-data) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
   (let [^ISpout spout (:object task-data)
@@ -453,7 +451,7 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-acked-tuple! (:stats executor-data) (:acked-counter executor-data) (:stream tuple-info) time-delta))))
+      (stats/spout-acked-tuple! (:stats executor-data) (StormMetricRegistry/counter "acked" worker-context (:component-id executor-data) (:executor-id executor-data) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)

http://git-wip-us.apache.org/repos/asf/storm/blob/c91da676/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index a2f6c54..edc144c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -130,8 +130,7 @@
         stream->component->grouper (:stream->component->grouper executor-data)
         user-context (:user-context task-data)
         executor-stats (:stats executor-data)
-        debug? (= true (storm-conf TOPOLOGY-DEBUG))
-        ^Counter emitted-counter (StormMetricRegistry/counter "emitted" worker-context component-id)]
+        debug? (= true (storm-conf TOPOLOGY-DEBUG))]
         
     (fn ([^Integer out-task-id ^String stream ^List values]
           (when debug?
@@ -144,7 +143,7 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (stats/emitted-tuple! executor-stats emitted-counter stream)
+              (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (:executor-id executor-data) stream) stream)
               (if out-task-id
                 (stats/transferred-tuples! executor-stats stream 1)))
             (if out-task-id [out-task-id])
@@ -164,7 +163,7 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (stats/emitted-tuple! executor-stats emitted-counter stream)
+               (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (:executor-id executor-data) stream) stream)
                (stats/transferred-tuples! executor-stats stream (count out-tasks)))
              out-tasks)))
     ))

http://git-wip-us.apache.org/repos/asf/storm/blob/c91da676/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 200ddcf..2bab4e9 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -68,20 +68,20 @@ public class StormMetricRegistry {
         );
     }
 
-    public static Meter meter(String name, WorkerTopologyContext context, String componentId){
-        String metricName = metricName(name, context.getStormId(), componentId, context.getThisWorkerPort());
+    public static Meter meter(String name, WorkerTopologyContext context, String componentId, String executorId, String streamId){
+        String metricName = metricName(name, context.getStormId(), componentId, streamId,executorId, context.getThisWorkerPort());
         return REGISTRY.meter(metricName);
     }
 
-    public static Counter counter(String name, WorkerTopologyContext context, String componentId){
-        String metricName = metricName(name, context.getStormId(), componentId, context.getThisWorkerPort());
+    public static Counter counter(String name, WorkerTopologyContext context, String componentId, String executorId, String streamId){
+        String metricName = metricName(name, context.getStormId(), componentId, streamId,executorId, context.getThisWorkerPort());
         return REGISTRY.counter(metricName);
     }
 
     public static void start(Map<String, Object> stormConfig, DaemonType type){
         String localHost = "localhost";
         try {
-            hostName = Utils.localHostname();
+            hostName = dotToUnderScore(Utils.localHostname());
         } catch (UnknownHostException e) {
              LOG.warn("Unable to determine hostname while starting the metrics system. Hostname will be reported" +
                      " as 'localhost'.");
@@ -130,9 +130,27 @@ public class StormMetricRegistry {
         }
     }
 
-    public static String metricName(String name, String stormId, String componentId, Integer workerPort){
-        return String.format("storm.worker.%s.%s.%s.%s-%s", stormId, hostName, componentId, workerPort, name);
+    public static String metricName(String name, String stormId, String componentId, String streamId, String executorId, Integer workerPort){
+        return String.format("storm.worker.%s.%s.%s.%s.%s.%s-%s",
+                stormId,
+                hostName,
+                dotToUnderScore(componentId),
+                dotToUnderScore(streamId),
+                dotToUnderScore(executorId),
+                workerPort,
+                name);
     }
 
+    public static String metricName(String name, String stormId, String componentId, Integer workerPort){
+        return String.format("storm.worker.%s.%s.%s.%s-%s",
+                stormId,
+                hostName,
+                dotToUnderScore(componentId),
+                workerPort,
+                name);
+    }
 
+    private static String dotToUnderScore(String str){
+        return str.replace('.', '_');
+    }
 }


[32/38] storm git commit: STORM-2153: remove unnecessary Metrics2Utils class

Posted by pt...@apache.org.
STORM-2153: remove unnecessary Metrics2Utils class


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

Branch: refs/heads/1.x-branch
Commit: 6cb81fd7a911ddcb986d332ec7e793acd009f0a9
Parents: eafc003
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Jan 11 15:59:55 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Jan 11 15:59:55 2018 -0500

----------------------------------------------------------------------
 .../apache/storm/metrics2/Metrics2Utils.java    | 28 --------------------
 .../reporters/ScheduledStormReporter.java       |  9 ++-----
 2 files changed, 2 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/6cb81fd7/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java b/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java
deleted file mode 100644
index 716b8b7..0000000
--- a/storm-core/src/jvm/org/apache/storm/metrics2/Metrics2Utils.java
+++ /dev/null
@@ -1,28 +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 org.apache.storm.metrics2;
-
-public class Metrics2Utils {
-    private Metrics2Utils(){}
-
-    public static Object instantiate(String klass) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
-        Class<?> c = Class.forName(klass);
-        return  c.newInstance();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/6cb81fd7/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
index b7ffa61..dccba06 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
@@ -18,7 +18,6 @@
 package org.apache.storm.metrics2.reporters;
 
 import com.codahale.metrics.ScheduledReporter;
-import org.apache.storm.metrics2.Metrics2Utils;
 import org.apache.storm.metrics2.filters.StormMetricsFilter;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
@@ -77,12 +76,8 @@ public abstract class ScheduledStormReporter implements StormReporter{
         if(filterConf != null) {
             String clazz = (String) filterConf.get("class");
             if (clazz != null) {
-                try {
-                    filter = (StormMetricsFilter) Metrics2Utils.instantiate(clazz);
-                    filter.prepare(filterConf);
-                } catch (Exception e) {
-                    throw new RuntimeException("Unable to instantiate StormMetricsFilter class: " + clazz);
-                }
+                filter = Utils.newInstance(clazz);
+                filter.prepare(filterConf);
             }
         }
         return filter;


[24/38] storm git commit: STORM-2153: Replace timer with ScheduledThreadPoolExecutor

Posted by pt...@apache.org.
STORM-2153: Replace timer with ScheduledThreadPoolExecutor


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

Branch: refs/heads/1.x-branch
Commit: e13f903452585ab84e08d5a7ac3a79c43141f232
Parents: 868de5b
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Jan 3 14:48:42 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Jan 3 14:48:42 2018 -0500

----------------------------------------------------------------------
 .../jvm/org/apache/storm/utils/DisruptorQueue.java   | 15 +++++++--------
 1 file changed, 7 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e13f9034/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
index d7cf401..6ea3683 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
@@ -48,6 +48,7 @@ import java.util.TimerTask;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -65,7 +66,7 @@ public class DisruptorQueue implements IStatefulObject {
     private static final Object INTERRUPT = new Object();
     private static final String PREFIX = "disruptor-";
     private static final FlusherPool FLUSHER = new FlusherPool();
-    private static final Timer METRICS_TIMER = new Timer("disruptor-metrics-timer", true);
+    private static final ScheduledThreadPoolExecutor METRICS_REPORTER_EXECUTOR = new ScheduledThreadPoolExecutor(1);
 
     private static int getNumFlusherPoolThreads() {
         int numThreads = 100;
@@ -437,17 +438,15 @@ public class DisruptorQueue implements IStatefulObject {
 
         _flusher = new Flusher(Math.max(flushInterval, 1), _queueName);
         _flusher.start();
-        try {
-            METRICS_TIMER.schedule(new TimerTask() {
+        if(!METRICS_REPORTER_EXECUTOR.isShutdown()) {
+            METRICS_REPORTER_EXECUTOR.scheduleAtFixedRate(new Runnable() {
                 @Override
                 public void run() {
                     _disruptorMetrics.set(_metrics);
                 }
-            }, 15000, 15000);
-        } catch (IllegalStateException e){
-            // Ignore. IllegalStateException is thrown by Timer.schedule() if the timer
-            // has been cancelled. (This happens in unit tests)
+            }, 15, 15, TimeUnit.SECONDS);
         }
+
     }
 
     public String getName() {
@@ -463,7 +462,7 @@ public class DisruptorQueue implements IStatefulObject {
             publishDirect(new ArrayList<Object>(Arrays.asList(INTERRUPT)), true);
             _flusher.close();
             _metrics.close();
-            METRICS_TIMER.cancel();
+            METRICS_REPORTER_EXECUTOR.shutdown();
         } catch (InsufficientCapacityException e) {
             //This should be impossible
             throw new RuntimeException(e);


[15/38] storm git commit: STORM-2153 New Metrics Reporting API

Posted by pt...@apache.org.
STORM-2153 New Metrics Reporting API

* address missing sampling rate
* rename field names cause we use Counter instead of Meter


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

Branch: refs/heads/1.x-branch
Commit: 00a382b017c1e29863ac4d9a4449086ef79384e4
Parents: 85dbacd
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Thu Nov 30 10:38:27 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Thu Nov 30 10:41:13 2017 +0900

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    | 22 ++++------
 .../src/clj/org/apache/storm/daemon/task.clj    |  8 ++--
 storm-core/src/clj/org/apache/storm/stats.clj   | 43 ++++++++++++--------
 3 files changed, 39 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/00a382b0/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 720bfa7..0aca4bd 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -257,7 +257,7 @@
      :batch-transfer-queue batch-transfer->worker
      :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
      :suicide-fn (:suicide-fn worker)
-     :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker) 
+     :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker)
                                                           :acls (Utils/getWorkerACL storm-conf)
                                                           :context (ClusterStateContext. DaemonType/WORKER))
      :type executor-type
@@ -280,8 +280,8 @@
                                (log-message "Got interrupted excpetion shutting thread down...")
                                ((:suicide-fn <>))))
      :sampler (mk-stats-sampler storm-conf)
-     :failed-meter (StormMetricRegistry/counter "failed" worker-context component-id)
-     :acked-meter (StormMetricRegistry/counter "acked" worker-context component-id)
+     :failed-counter (StormMetricRegistry/counter "failed" worker-context component-id)
+     :acked-counter (StormMetricRegistry/counter "acked" worker-context component-id)
      :spout-throttling-metrics (if (= executor-type :spout)
                                 (builtin-metrics/make-spout-throttling-data)
                                 nil)
@@ -437,27 +437,23 @@
 (defn- fail-spout-msg [executor-data task-data msg-id tuple-info time-delta reason id debug?]
   (let [^ISpout spout (:object task-data)
         storm-conf (:storm-conf executor-data)
-        task-id (:task-id task-data)
-        failed-meter (:failed-meter executor-data)]
+        task-id (:task-id task-data)]
     ;;TODO: need to throttle these when there's lots of failures
     (when debug?
       (log-message "SPOUT Failing " id ": " tuple-info " REASON: " reason " MSG-ID: " msg-id))
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (.inc ^Counter failed-meter)
-      (stats/spout-failed-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
+      (stats/spout-failed-tuple! (:stats executor-data) (:failed-counter executor-data) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
   (let [^ISpout spout (:object task-data)
-        task-id (:task-id task-data)
-        acked-meter (:acked-meter executor-data)]
+        task-id (:task-id task-data)]
     (when debug? (log-message "SPOUT Acking message " id " " msg-id))
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (.inc ^Counter acked-meter)
-      (stats/spout-acked-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
+      (stats/spout-acked-tuple! (:stats executor-data) (:acked-counter executor-data) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)
@@ -825,8 +821,8 @@
                              (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when (<= 0 delta)
-                             (.inc ^Counter (:acked-meter (:executor-data task-data)))
                              (stats/bolt-acked-tuple! executor-stats
+                                                      (:acked-counter (:executor-data task-data))
                                                       (.getSourceComponent tuple)
                                                       (.getSourceStreamId tuple)
                                                       delta))))
@@ -841,8 +837,8 @@
                              (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when (<= 0 delta)
-                             (.inc  ^Counter (:failed-meter (:executor-data task-data)))
                              (stats/bolt-failed-tuple! executor-stats
+                                                       (:failed-counter (:executor-data task-data))
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)
                                                        delta))))

http://git-wip-us.apache.org/repos/asf/storm/blob/00a382b0/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index c43d20d..a2f6c54 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -131,7 +131,7 @@
         user-context (:user-context task-data)
         executor-stats (:stats executor-data)
         debug? (= true (storm-conf TOPOLOGY-DEBUG))
-        ^Counter emitted-meter (StormMetricRegistry/counter "emitted" worker-context component-id)]
+        ^Counter emitted-counter (StormMetricRegistry/counter "emitted" worker-context component-id)]
         
     (fn ([^Integer out-task-id ^String stream ^List values]
           (when debug?
@@ -144,8 +144,7 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (.inc ^Counter emitted-meter)
-              (stats/emitted-tuple! executor-stats stream)
+              (stats/emitted-tuple! executor-stats emitted-counter stream)
               (if out-task-id
                 (stats/transferred-tuples! executor-stats stream 1)))
             (if out-task-id [out-task-id])
@@ -165,8 +164,7 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (.inc ^Counter emitted-meter)
-               (stats/emitted-tuple! executor-stats stream)
+               (stats/emitted-tuple! executor-stats emitted-counter stream)
                (stats/transferred-tuples! executor-stats stream (count out-tasks)))
              out-tasks)))
     ))

http://git-wip-us.apache.org/repos/asf/storm/blob/00a382b0/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
index 17d0219..41aaf04 100644
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@ -26,7 +26,8 @@
             WorkerResources])
   (:import [org.apache.storm.utils Utils])
   (:import [org.apache.storm.scheduler WorkerSlot])
-  (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
+  (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric]
+           (com.codahale.metrics Counter))
   (:use [org.apache.storm log util])
   (:use [clojure.math.numeric-tower :only [ceil]]))
 
@@ -117,9 +118,11 @@
   `(:complete-latencies ~stats))
 
 (defn emitted-tuple!
-  [stats stream]
-  (let [^MultiCountStatAndMetric emitted (stats-emitted stats)]
-    (.incBy emitted ^Object stream ^long (stats-rate stats))))
+  [stats ^Counter emitted-counter stream]
+  (let [^MultiCountStatAndMetric emitted (stats-emitted stats)
+        ^long rate (stats-rate stats)]
+    (.incBy emitted ^Object stream rate)
+    (.inc emitted-counter rate)))
 
 (defn transferred-tuples!
   [stats stream amt]
@@ -135,30 +138,38 @@
     (.record exec-lat key latency-ms)))
 
 (defn bolt-acked-tuple!
-  [^BoltExecutorStats stats component stream latency-ms]
+  [^BoltExecutorStats stats ^Counter acked-counter component stream latency-ms]
   (let [key [component stream]
         ^MultiCountStatAndMetric acked (stats-acked stats)
-        ^MultiLatencyStatAndMetric process-lat (stats-process-latencies stats)]
-    (.incBy acked key (stats-rate stats))
+        ^MultiLatencyStatAndMetric process-lat (stats-process-latencies stats)
+        ^long rate (stats-rate stats)]
+    (.incBy acked key rate)
+    (.inc acked-counter rate)
     (.record process-lat key latency-ms)))
 
 (defn bolt-failed-tuple!
-  [^BoltExecutorStats stats component stream latency-ms]
+  [^BoltExecutorStats stats ^Counter failed-counter component stream latency-ms]
   (let [key [component stream]
-        ^MultiCountStatAndMetric failed (stats-failed stats)]
-    (.incBy failed key (stats-rate stats))))
+        ^MultiCountStatAndMetric failed (stats-failed stats)
+        ^long rate (stats-rate stats)]
+    (.incBy failed key rate)
+    (.inc failed-counter rate)))
 
 (defn spout-acked-tuple!
-  [^SpoutExecutorStats stats stream latency-ms]
+  [^SpoutExecutorStats stats ^Counter acked-counter stream latency-ms]
   (let [^MultiCountStatAndMetric acked (stats-acked stats)
-        ^MultiLatencyStatAndMetric complete-latencies (stats-complete-latencies stats)]
-    (.incBy acked stream (stats-rate stats))
+        ^MultiLatencyStatAndMetric complete-latencies (stats-complete-latencies stats)
+        ^long rate (stats-rate stats)]
+    (.incBy acked stream rate)
+    (.inc acked-counter rate)
     (.record complete-latencies stream latency-ms)))
 
 (defn spout-failed-tuple!
-  [^SpoutExecutorStats stats stream latency-ms]
-  (let [^MultiCountStatAndMetric failed (stats-failed stats)]
-    (.incBy failed stream (stats-rate stats))))
+  [^SpoutExecutorStats stats ^Counter failed-counter stream latency-ms]
+  (let [^MultiCountStatAndMetric failed (stats-failed stats)
+        ^long rate (stats-rate stats)]
+    (.incBy failed stream rate)
+    (.inc failed-counter rate)))
 
 (defn- close-stat! [stat]
   (.close stat))


[21/38] storm git commit: STORM-2153: Add transferred metric

Posted by pt...@apache.org.
STORM-2153: Add transferred metric


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

Branch: refs/heads/1.x-branch
Commit: 47be75c8d2fc5f9c7901595fecb385393f9fab8b
Parents: 5ce45b7
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Dec 15 12:48:08 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Dec 15 12:48:08 2017 -0500

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/task.clj | 4 ++--
 storm-core/src/clj/org/apache/storm/stats.clj       | 3 ++-
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/47be75c8/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 8eb432e..7132fc1 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -145,7 +145,7 @@
             (when (emit-sampler)
               (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream)
               (if out-task-id
-                (stats/transferred-tuples! executor-stats stream 1)))
+                (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
@@ -164,7 +164,7 @@
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
                (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream)
-               (stats/transferred-tuples! executor-stats stream (count out-tasks)))
+               (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream (count out-tasks)))
              out-tasks)))
     ))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/47be75c8/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
index 41aaf04..85c7bbe 100644
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@ -125,8 +125,9 @@
     (.inc emitted-counter rate)))
 
 (defn transferred-tuples!
-  [stats stream amt]
+  [stats ^Counter transferred-counter stream amt]
   (let [^MultiCountStatAndMetric transferred (stats-transferred stats)]
+    (.inc transferred-counter amt)
     (.incBy transferred ^Object stream ^long (* (stats-rate stats) amt))))
 
 (defn bolt-execute-tuple!


[35/38] storm git commit: STORM-2153: fix a missing spot, extract string to constant, modify accessors

Posted by pt...@apache.org.
STORM-2153: fix a missing spot, extract string to constant, modify accessors


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

Branch: refs/heads/1.x-branch
Commit: 7947a0755c6717de16a7755ec7f1f2dc83388d11
Parents: b257ba4
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Fri Jan 12 12:56:22 2018 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Fri Jan 12 12:56:22 2018 +0900

----------------------------------------------------------------------
 .../org/apache/storm/metrics2/TaskMetrics.java  | 23 ++++++++++++--------
 1 file changed, 14 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/7947a075/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
index 05c62da..239c1a0 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
@@ -26,10 +26,15 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 public class TaskMetrics {
-    ConcurrentMap<String, Counter> ackedByStream = new ConcurrentHashMap<>();
-    ConcurrentMap<String, Counter> failedByStream = new ConcurrentHashMap<>();
-    ConcurrentMap<String, Counter> emittedByStream = new ConcurrentHashMap<>();
-    ConcurrentMap<String, Counter> transferredByStream = new ConcurrentHashMap<>();
+    private static final String METRIC_NAME_ACKED = "acked";
+    private static final String METRIC_NAME_FAILED = "failed";
+    private static final String METRIC_NAME_EMITTED = "emitted";
+    private static final String METRIC_NAME_TRANSFERRED = "transferred";
+
+    private ConcurrentMap<String, Counter> ackedByStream = new ConcurrentHashMap<>();
+    private ConcurrentMap<String, Counter> failedByStream = new ConcurrentHashMap<>();
+    private ConcurrentMap<String, Counter> emittedByStream = new ConcurrentHashMap<>();
+    private ConcurrentMap<String, Counter> transferredByStream = new ConcurrentHashMap<>();
 
     private String topologyId;
     private String componentId;
@@ -46,16 +51,16 @@ public class TaskMetrics {
     public Counter getAcked(String streamId) {
         Counter c = this.ackedByStream.get(streamId);
         if (c == null) {
-            c = StormMetricRegistry.counter("acked", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            c = StormMetricRegistry.counter(METRIC_NAME_ACKED, this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
             this.ackedByStream.put(streamId, c);
         }
         return c;
     }
 
     public Counter getFailed(String streamId) {
-        Counter c = this.ackedByStream.get(streamId);
+        Counter c = this.failedByStream.get(streamId);
         if (c == null) {
-            c = StormMetricRegistry.counter("failed", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            c = StormMetricRegistry.counter(METRIC_NAME_FAILED, this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
             this.failedByStream.put(streamId, c);
         }
         return c;
@@ -64,7 +69,7 @@ public class TaskMetrics {
     public Counter getEmitted(String streamId) {
         Counter c = this.emittedByStream.get(streamId);
         if (c == null) {
-            c = StormMetricRegistry.counter("emitted", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            c = StormMetricRegistry.counter(METRIC_NAME_EMITTED, this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
             this.emittedByStream.put(streamId, c);
         }
         return c;
@@ -73,7 +78,7 @@ public class TaskMetrics {
     public Counter getTransferred(String streamId) {
         Counter c = this.transferredByStream.get(streamId);
         if (c == null) {
-            c = StormMetricRegistry.counter("transferred", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            c = StormMetricRegistry.counter(METRIC_NAME_TRANSFERRED, this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
             this.transferredByStream.put(streamId, c);
         }
         return c;


[09/38] storm git commit: address review comments

Posted by pt...@apache.org.
address review comments


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

Branch: refs/heads/1.x-branch
Commit: cfa6cd63158db66a951f804ded75575b2b4f3d4a
Parents: de20cbd
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Sep 29 13:58:41 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Sep 29 13:58:41 2017 -0400

----------------------------------------------------------------------
 .../apache/storm/starter/AnchoredWordCount.java | 12 +++++------
 .../storm/metrics2/StormMetricRegistry.java     | 21 +++++++++-----------
 .../reporters/ConsoleStormReporter.java         |  2 +-
 .../metrics2/reporters/CsvStormReporter.java    |  2 +-
 .../reporters/GangliaStormReporter.java         |  3 +--
 .../reporters/GraphiteStormReporter.java        |  4 +---
 .../metrics2/reporters/JmxStormReporter.java    |  2 +-
 .../reporters/ScheduledStormReporter.java       | 12 +++++------
 .../storm/metrics2/reporters/StormReporter.java |  2 +-
 9 files changed, 27 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java
index 3b22c9f..c56473a 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/AnchoredWordCount.java
@@ -43,14 +43,14 @@ public class AnchoredWordCount {
     public static class RandomSentenceSpout extends BaseRichSpout {
         private static final Logger LOG = LoggerFactory.getLogger(RandomSentenceSpout.class);
 
-        SpoutOutputCollector _collector;
-        Random _rand;
+        SpoutOutputCollector collector;
+        Random random;
 
 
         @Override
         public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) {
-            _collector = collector;
-            _rand = new Random();
+            this.collector = collector;
+            this.random = new Random();
         }
 
         @Override
@@ -58,9 +58,9 @@ public class AnchoredWordCount {
             Utils.sleep(10);
             String[] sentences = new String[]{sentence("the cow jumped over the moon"), sentence("an apple a day keeps the doctor away"),
                     sentence("four score and seven years ago"), sentence("snow white and the seven dwarfs"), sentence("i am at two with nature")};
-            final String sentence = sentences[_rand.nextInt(sentences.length)];
+            final String sentence = sentences[random.nextInt(sentences.length)];
 
-            _collector.emit(new Values(sentence), UUID.randomUUID());
+            this.collector.emit(new Values(sentence), UUID.randomUUID());
         }
 
         protected String sentence(String input) {

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index a3b0db9..60d4191 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -23,6 +23,7 @@ import org.apache.storm.Config;
 import org.apache.storm.cluster.DaemonType;
 import org.apache.storm.metrics2.reporters.StormReporter;
 import org.apache.storm.task.WorkerTopologyContext;
+import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,8 +47,8 @@ public class StormMetricRegistry {
     public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, String componentId, Integer port){
         SimpleGauge<T> gauge = new SimpleGauge<>(initialValue);
         String metricName = metricName(name, topologyId, componentId, port);
-            if(REGISTRY.getGauges().containsKey(metricName)){
-                return (SimpleGauge)REGISTRY.getGauges().get(metricName);
+        if(REGISTRY.getGauges().containsKey(metricName)){
+            return (SimpleGauge)REGISTRY.getGauges().get(metricName);
         } else {
             return REGISTRY.register(metricName, gauge);
         }
@@ -72,16 +73,12 @@ public class StormMetricRegistry {
     }
 
     public static void start(Map<String, Object> stormConfig, DaemonType type){
-        String localHost = (String)stormConfig.get(Config.STORM_LOCAL_HOSTNAME);
-        if(localHost != null){
-            hostName = localHost;
-        } else {
-            try {
-                hostName = InetAddress.getLocalHost().getCanonicalHostName();
-            } catch (UnknownHostException e) {
-                 LOG.warn("Unable to determine hostname while starting the metrics system. Hostname ill be reported" +
-                         " as 'localhost'.");
-            }
+        String localHost = "localhost";
+        try {
+            hostName = Utils.localHostname();
+        } catch (UnknownHostException e) {
+             LOG.warn("Unable to determine hostname while starting the metrics system. Hostname ill be reported" +
+                     " as 'localhost'.");
         }
 
         LOG.info("Starting metrics reporters...");

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
index abb5226..4c91f03 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ConsoleStormReporter.java
@@ -28,7 +28,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class ConsoleStormReporter extends ScheduledStormReporter<ConsoleReporter> {
+public class ConsoleStormReporter extends ScheduledStormReporter {
     private final static Logger LOG = LoggerFactory.getLogger(ConsoleStormReporter.class);
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
index 24c6eed..5d9ff4e 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/CsvStormReporter.java
@@ -31,7 +31,7 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class CsvStormReporter extends ScheduledStormReporter<CsvReporter> {
+public class CsvStormReporter extends ScheduledStormReporter {
     private final static Logger LOG = LoggerFactory.getLogger(CsvStormReporter.class);
 
     public static final String CSV_LOG_DIR = "csv.log.dir";

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
index e7dc5f4..09af2e1 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GangliaStormReporter.java
@@ -30,7 +30,7 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class GangliaStormReporter extends ScheduledStormReporter<GangliaReporter> {
+public class GangliaStormReporter extends ScheduledStormReporter {
     private final static Logger LOG = LoggerFactory.getLogger(GangliaStormReporter.class);
 
     public static final String GANGLIA_HOST = "ganglia.host";
@@ -99,7 +99,6 @@ public class GangliaStormReporter extends ScheduledStormReporter<GangliaReporter
             GMetric sender = new GMetric(group, port, mode, ttl);
             reporter = builder.build(sender);
         }catch (IOException ioe){
-            //TODO
             LOG.error("Exception in GangliaReporter config", ioe);
         }
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
index 0f88fc4..ba3c0c5 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/GraphiteStormReporter.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class GraphiteStormReporter extends ScheduledStormReporter<GraphiteReporter> {
+public class GraphiteStormReporter extends ScheduledStormReporter {
     private final static Logger LOG = LoggerFactory.getLogger(GraphiteStormReporter.class);
 
     public static final String GRAPHITE_PREFIXED_WITH = "graphite.prefixed.with";
@@ -76,11 +76,9 @@ public class GraphiteStormReporter extends ScheduledStormReporter<GraphiteReport
         Integer port = getMetricsTargetPort(reporterConf);
         String transport = getMetricsTargetTransport(reporterConf);
         GraphiteSender sender = null;
-        //TODO: error checking
         if (transport.equalsIgnoreCase("udp")) {
             sender = new GraphiteUDP(host, port);
         } else {
-            //TODO: pickled support
             sender = new Graphite(host, port);
         }
         reporter = builder.build(sender);

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
index 5b932ea..325ab1d 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/JmxStormReporter.java
@@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public class JmxStormReporter implements StormReporter<JmxReporter> {
+public class JmxStormReporter implements StormReporter {
     private final static Logger LOG = LoggerFactory.getLogger(JmxStormReporter.class);
     public static final String JMX_DOMAIN = "jmx.domain";
     JmxReporter reporter = null;

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
index 940cb19..6ef39b6 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
@@ -27,11 +27,11 @@ import org.slf4j.LoggerFactory;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-public abstract class ScheduledStormReporter<T extends ScheduledReporter> implements StormReporter{
+public abstract class ScheduledStormReporter implements StormReporter{
     private static final Logger LOG = LoggerFactory.getLogger(ScheduledStormReporter.class);
     protected ScheduledReporter reporter;
-    long reportingPeriod;
-    TimeUnit reportingPeriodUnit;
+    protected long reportingPeriod;
+    protected TimeUnit reportingPeriodUnit;
 
     @Override
     public void start() {
@@ -54,7 +54,7 @@ public abstract class ScheduledStormReporter<T extends ScheduledReporter> implem
     }
 
 
-    static TimeUnit getReportPeriodUnit(Map<String, Object> reporterConf) {
+    public static TimeUnit getReportPeriodUnit(Map<String, Object> reporterConf) {
         TimeUnit unit = getTimeUnitForConfig(reporterConf, REPORT_PERIOD_UNITS);
         return unit == null ? TimeUnit.SECONDS : unit;
     }
@@ -67,11 +67,11 @@ public abstract class ScheduledStormReporter<T extends ScheduledReporter> implem
         return null;
     }
 
-    static long getReportPeriod(Map reporterConf) {
+    public static long getReportPeriod(Map reporterConf) {
         return Utils.getInt(reporterConf.get(REPORT_PERIOD), 10).longValue();
     }
 
-    static StormMetricsFilter getMetricsFilter(Map reporterConf){
+    public static StormMetricsFilter getMetricsFilter(Map reporterConf){
         StormMetricsFilter filter = null;
         Map<String, Object> filterConf = (Map)reporterConf.get("filter");
         String clazz = (String) filterConf.get("class");

http://git-wip-us.apache.org/repos/asf/storm/blob/cfa6cd63/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java
index c36e44e..a5d9798 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/StormReporter.java
@@ -22,7 +22,7 @@ import com.codahale.metrics.Reporter;
 
 import java.util.Map;
 
-public interface StormReporter<T extends Reporter> {
+public interface StormReporter extends Reporter {
     String REPORT_PERIOD = "report.period";
     String REPORT_PERIOD_UNITS = "report.period.units";
 


[31/38] storm git commit: STORM-2153: Add missing license header

Posted by pt...@apache.org.
STORM-2153: Add missing license header


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

Branch: refs/heads/1.x-branch
Commit: eafc003794310f6e69c09dc7fd23f9cc5d5aaeff
Parents: fea11b2
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Jan 11 15:55:59 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Jan 11 15:55:59 2018 -0500

----------------------------------------------------------------------
 .../jvm/org/apache/storm/metrics2/TaskMetrics.java | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/eafc0037/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
index 5bb01d2..550b176 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
@@ -1,3 +1,20 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.storm.metrics2;
 
 import com.codahale.metrics.Counter;


[07/38] storm git commit: revert pom formatting

Posted by pt...@apache.org.
revert pom formatting


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

Branch: refs/heads/1.x-branch
Commit: a169356fac2bb4a07ddc4372fe04112880d9f14b
Parents: 57a50f3
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Aug 30 17:14:31 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Aug 30 17:14:31 2017 -0400

----------------------------------------------------------------------
 external/storm-autocreds/pom.xml | 26 ++++++++++++++------------
 1 file changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a169356f/external/storm-autocreds/pom.xml
----------------------------------------------------------------------
diff --git a/external/storm-autocreds/pom.xml b/external/storm-autocreds/pom.xml
index c05b620..1dbdc0a 100644
--- a/external/storm-autocreds/pom.xml
+++ b/external/storm-autocreds/pom.xml
@@ -15,7 +15,9 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
         <artifactId>storm</artifactId>
         <groupId>org.apache.storm</groupId>
@@ -124,12 +126,12 @@
         </dependency>
     </dependencies>
     <build>
-    <plugins>
-       <plugin>
-          <groupId>org.codehaus.mojo</groupId>
-          <artifactId>appassembler-maven-plugin</artifactId>
-          <version>1.9</version>
-          <executions>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>appassembler-maven-plugin</artifactId>
+                <version>1.9</version>
+                <executions>
                     <execution>
                         <id>create-repo</id>
                         <goals>
@@ -140,8 +142,8 @@
                             <repositoryLayout>flat</repositoryLayout>
                         </configuration>
                     </execution>
-          </executions>
-        </plugin>
-    </plugins>
-   </build>
-</project>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+</project>
\ No newline at end of file


[25/38] storm git commit: STORM-2153: set disruptor reporter threads to daemon

Posted by pt...@apache.org.
STORM-2153: set disruptor reporter threads to daemon


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

Branch: refs/heads/1.x-branch
Commit: 8af4fcac3267b8468c7a4cb02afbe9b7fd083d4b
Parents: e13f903
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Jan 3 15:23:38 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Jan 3 15:23:38 2018 -0500

----------------------------------------------------------------------
 storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/8af4fcac/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
index 6ea3683..d7497d6 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
@@ -66,7 +66,8 @@ public class DisruptorQueue implements IStatefulObject {
     private static final Object INTERRUPT = new Object();
     private static final String PREFIX = "disruptor-";
     private static final FlusherPool FLUSHER = new FlusherPool();
-    private static final ScheduledThreadPoolExecutor METRICS_REPORTER_EXECUTOR = new ScheduledThreadPoolExecutor(1);
+    private static final ScheduledThreadPoolExecutor METRICS_REPORTER_EXECUTOR = new ScheduledThreadPoolExecutor(1,
+            new ThreadFactoryBuilder().setDaemon(true).setNameFormat(PREFIX + "metrics-reporter").build());
 
     private static int getNumFlusherPoolThreads() {
         int numThreads = 100;


[19/38] storm git commit: STORM-2153: fix test failures

Posted by pt...@apache.org.
STORM-2153: fix test failures


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

Branch: refs/heads/1.x-branch
Commit: cd272c41521c17adde31b64471b748917dbe5ab0
Parents: c91da67
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Dec 13 17:10:48 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Dec 13 17:10:48 2017 -0500

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/executor.clj | 8 ++++----
 storm-core/src/clj/org/apache/storm/daemon/task.clj     | 4 ++--
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/cd272c41/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index fa7d44c..993add6 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -442,7 +442,7 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-failed-tuple! (:stats executor-data) (StormMetricRegistry/counter "failed" worker-context (:component-id executor-data) (:executor-id executor-data) (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-failed-tuple! (:stats executor-data) (StormMetricRegistry/counter "failed" (:worker-context executor-data) (:component-id executor-data) (pr-str (:executor-id executor-data)) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
   (let [^ISpout spout (:object task-data)
@@ -451,7 +451,7 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-acked-tuple! (:stats executor-data) (StormMetricRegistry/counter "acked" worker-context (:component-id executor-data) (:executor-id executor-data) (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-acked-tuple! (:stats executor-data) (StormMetricRegistry/counter "acked" (:worker-context executor-data) (:component-id executor-data) (pr-str (:executor-id executor-data)) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)
@@ -820,7 +820,7 @@
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-acked-tuple! executor-stats
-                                                      (:acked-counter (:executor-data task-data))
+                                                      (StormMetricRegistry/counter "acked" worker-context  (:component-id executor-data) (pr-str (:executor-id executor-data)) (.getSourceStreamId tuple))
                                                       (.getSourceComponent tuple)
                                                       (.getSourceStreamId tuple)
                                                       delta))))
@@ -836,7 +836,7 @@
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-failed-tuple! executor-stats
-                                                       (:failed-counter (:executor-data task-data))
+                                                       (StormMetricRegistry/counter "failed" worker-context (:component-id executor-data) (pr-str (:executor-id executor-data)) (.getSourceStreamId tuple))
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)
                                                        delta))))

http://git-wip-us.apache.org/repos/asf/storm/blob/cd272c41/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index edc144c..8eb432e 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -143,7 +143,7 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (:executor-id executor-data) stream) stream)
+              (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream)
               (if out-task-id
                 (stats/transferred-tuples! executor-stats stream 1)))
             (if out-task-id [out-task-id])
@@ -163,7 +163,7 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (:executor-id executor-data) stream) stream)
+               (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id (pr-str (:executor-id executor-data)) stream) stream)
                (stats/transferred-tuples! executor-stats stream (count out-tasks)))
              out-tasks)))
     ))


[26/38] storm git commit: STORM-2152: address additional review comments

Posted by pt...@apache.org.
STORM-2152: address additional review comments


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

Branch: refs/heads/1.x-branch
Commit: 44cd8ac3b4b55cfa86b63d45db2c0407f4c26417
Parents: 8af4fca
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Jan 9 15:07:16 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Jan 9 15:07:16 2018 -0500

----------------------------------------------------------------------
 .../storm/metrics2/StormMetricRegistry.java     | 20 ++++++++++++--------
 .../org/apache/storm/task/TopologyContext.java  |  2 +-
 2 files changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/44cd8ac3/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 2bab4e9..789367b 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -23,6 +23,7 @@ import com.codahale.metrics.MetricRegistry;
 import org.apache.storm.Config;
 import org.apache.storm.cluster.DaemonType;
 import org.apache.storm.metrics2.reporters.StormReporter;
+import org.apache.storm.task.TopologyContext;
 import org.apache.storm.task.WorkerTopologyContext;
 import org.apache.storm.utils.Utils;
 import org.slf4j.Logger;
@@ -46,12 +47,11 @@ public class StormMetricRegistry {
     private static String hostName = null;
 
     public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, String componentId, Integer port){
-        SimpleGauge<T> gauge = new SimpleGauge<>(initialValue);
         String metricName = metricName(name, topologyId, componentId, port);
         if(REGISTRY.getGauges().containsKey(metricName)){
             return (SimpleGauge)REGISTRY.getGauges().get(metricName);
         } else {
-            return REGISTRY.register(metricName, gauge);
+            return REGISTRY.register(metricName, new SimpleGauge<>(initialValue));
         }
     }
 
@@ -79,7 +79,6 @@ public class StormMetricRegistry {
     }
 
     public static void start(Map<String, Object> stormConfig, DaemonType type){
-        String localHost = "localhost";
         try {
             hostName = dotToUnderScore(Utils.localHostname());
         } catch (UnknownHostException e) {
@@ -110,11 +109,7 @@ public class StormMetricRegistry {
         String clazz = (String)reporterConfig.get("class");
         StormReporter reporter = null;
         LOG.info("Attempting to instantiate reporter class: {}", clazz);
-        try{
-            reporter = (StormReporter)Metrics2Utils.instantiate(clazz);
-        } catch(Exception e){
-            LOG.warn("Unable to instantiate metrics reporter class: {}. Will skip this reporter.", clazz, e);
-        }
+        reporter = Utils.newInstance(clazz);
         if(reporter != null){
             reporter.prepare(REGISTRY, stormConfig, reporterConfig);
             reporter.start();
@@ -150,6 +145,15 @@ public class StormMetricRegistry {
                 name);
     }
 
+    public static String metricName(String name, TopologyContext context){
+        return String.format("storm.topology.%s.%s.%s.%s.%s-%s",
+                context.getStormId(),
+                hostName,
+                dotToUnderScore(context.getThisComponentId()),
+                context.getThisWorkerPort(),
+                name);
+    }
+
     private static String dotToUnderScore(String str){
         return str.replace('.', '_');
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/44cd8ac3/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
index 444a8a7..330fee1 100644
--- a/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
+++ b/storm-core/src/jvm/org/apache/storm/task/TopologyContext.java
@@ -410,6 +410,6 @@ public class TopologyContext extends WorkerTopologyContext implements IMetricsCo
     }
 
     private String metricName(String name){
-        return String.format("storm.topology.%s.%s.%s-%s", getStormId(), getThisComponentId(), getThisWorkerPort(), name);
+        return StormMetricRegistry.metricName(name, this);
     }
 }


[02/38] storm git commit: revert some unintended changes

Posted by pt...@apache.org.
revert some unintended changes


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

Branch: refs/heads/1.x-branch
Commit: 9a8dfb7d075eb0d0ca2fedd8df88cdb6608cb87e
Parents: fa1e59f
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Jul 11 14:54:46 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Jul 11 14:54:46 2017 -0400

----------------------------------------------------------------------
 .../src/jvm/org/apache/storm/starter/ExclamationTopology.java      | 2 +-
 .../src/jvm/org/apache/storm/starter/WordCountTopology.java        | 2 +-
 .../java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java   | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9a8dfb7d/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
index 9284b52..26e0430 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/ExclamationTopology.java
@@ -79,7 +79,7 @@ public class ExclamationTopology {
 
       LocalCluster cluster = new LocalCluster();
       cluster.submitTopology("test", conf, builder.createTopology());
-      Utils.sleep(100000);
+      Utils.sleep(10000);
       cluster.killTopology("test");
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8dfb7d/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
----------------------------------------------------------------------
diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
index 0611894..e4a5711 100644
--- a/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
+++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java
@@ -98,7 +98,7 @@ public class WordCountTopology {
       LocalCluster cluster = new LocalCluster();
       cluster.submitTopology("word-count", conf, builder.createTopology());
 
-      Thread.sleep(60000);
+      Thread.sleep(10000);
 
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8dfb7d/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
----------------------------------------------------------------------
diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
index 087aec5..2008a3e 100644
--- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
+++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/avro/ConfluentAvroSerializer.java
@@ -27,7 +27,7 @@ import java.io.IOException;
 import java.util.Map;
 
 /**
- * This class provides a mechanism to utilize the Confluent Schema StormMetricRegistry (https://github.com/confluentinc/schema-registry)
+ * This class provides a mechanism to utilize the Confluent Schema Registry (https://github.com/confluentinc/schema-registry)
  * for Storm to (de)serialize Avro generic records across a topology.  It assumes the schema registry is up and running
  * completely independent of Storm.
  */


[12/38] storm git commit: Merge branch '1.x-branch' into metrics_v2

Posted by pt...@apache.org.
Merge branch '1.x-branch' into metrics_v2


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

Branch: refs/heads/1.x-branch
Commit: 99bcf68449df63a83e124afaadaa61f61da15d22
Parents: a47b5fb 7962639
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Mon Nov 20 13:52:24 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Mon Nov 20 13:52:24 2017 -0500

----------------------------------------------------------------------
 bin/storm.py                                    |    1 -
 docs/Trident-API-Overview.md                    |    2 +-
 docs/storm-kafka-client.md                      |   46 +-
 .../storm/hbase/security/HBaseSecurityUtil.java |    2 +-
 .../state/TridentResultSetValuesMapper.java     |    6 +-
 .../apache/storm/druid/bolt/DruidBeamBolt.java  |   13 +-
 external/storm-eventhubs/pom.xml                |   11 -
 external/storm-hbase/README.md                  |   22 +-
 .../storm/hdfs/blobstore/HdfsBlobStore.java     |  107 +-
 .../storm/hdfs/blobstore/HdfsBlobStoreImpl.java |   14 +-
 .../storm/hdfs/bolt/AbstractHdfsBolt.java       |   48 +-
 .../org/apache/storm/hdfs/spout/HdfsSpout.java  | 1484 +++++++++---------
 .../hdfs/avro/TestFixedAvroSerializer.java      |   13 +-
 .../hdfs/avro/TestGenericAvroSerializer.java    |   11 +-
 .../storm/hdfs/blobstore/BlobStoreTest.java     |  897 ++++++-----
 .../hdfs/blobstore/HdfsBlobStoreImplTest.java   |  288 ++--
 .../hdfs/bolt/AvroGenericRecordBoltTest.java    |  142 +-
 .../apache/storm/hdfs/bolt/TestHdfsBolt.java    |  108 +-
 .../storm/hdfs/bolt/TestSequenceFileBolt.java   |   71 +-
 .../apache/storm/hdfs/spout/TestDirLock.java    |  281 ++--
 .../apache/storm/hdfs/spout/TestFileLock.java   |  679 ++++----
 .../storm/hdfs/spout/TestHdfsSemantics.java     |  311 ++--
 .../apache/storm/hdfs/spout/TestHdfsSpout.java  | 1235 +++++++--------
 .../storm/hdfs/spout/TestProgressTracker.java   |  179 +--
 .../storm/hdfs/testing/MiniDFSClusterRule.java  |   78 +
 .../apache/storm/jms/spout/JmsSpoutTest.java    |   72 +-
 .../apache/storm/kafka/spout/KafkaSpout.java    |  362 +++--
 .../storm/kafka/spout/KafkaSpoutConfig.java     |  167 +-
 .../KafkaSpoutRetryExponentialBackoff.java      |   32 +-
 .../kafka/spout/KafkaSpoutRetryService.java     |    6 +-
 .../apache/storm/kafka/spout/Subscription.java  |    5 +-
 .../kafka/spout/internal/OffsetManager.java     |  102 +-
 .../trident/KafkaTridentSpoutBatchMetadata.java |   78 +-
 .../spout/trident/KafkaTridentSpoutEmitter.java |  108 +-
 .../spout/trident/KafkaTridentSpoutManager.java |   10 +-
 .../spout/trident/KafkaTridentSpoutOpaque.java  |   18 +-
 .../KafkaTridentSpoutOpaqueCoordinator.java     |   26 +-
 .../spout/trident/TopicPartitionSerializer.java |   47 +
 .../storm/kafka/trident/TridentKafkaState.java  |    9 +
 .../storm/kafka/spout/KafkaSpoutCommitTest.java |  131 --
 .../storm/kafka/spout/KafkaSpoutConfigTest.java |   26 +-
 .../storm/kafka/spout/KafkaSpoutEmitTest.java   |  147 +-
 .../KafkaSpoutLogCompactionSupportTest.java     |  223 +++
 .../spout/KafkaSpoutMessagingGuaranteeTest.java |  207 +++
 .../kafka/spout/KafkaSpoutRebalanceTest.java    |   79 +-
 .../KafkaSpoutRetryExponentialBackoffTest.java  |  292 ++++
 .../kafka/spout/KafkaSpoutRetryLimitTest.java   |   27 +-
 .../kafka/spout/MaxUncommittedOffsetTest.java   |   64 +-
 .../kafka/spout/SingleTopicKafkaSpoutTest.java  |  153 +-
 .../spout/SingleTopicKafkaUnitSetupHelper.java  |   67 +
 .../SpoutWithMockedConsumerSetupHelper.java     |  136 +-
 .../SingleTopicKafkaSpoutConfiguration.java     |   25 +-
 .../kafka/spout/internal/OffsetManagerTest.java |  187 +++
 .../KafkaTridentSpoutBatchMetadataTest.java     |   66 +
 external/storm-kafka-monitor/pom.xml            |    6 -
 external/storm-kafka/pom.xml                    |   29 -
 .../apache/storm/kafka/PartitionManager.java    |   21 +-
 external/storm-kinesis/pom.xml                  |   11 -
 integration-test/README.md                      |    2 +-
 integration-test/config/Vagrantfile             |    8 +-
 integration-test/config/cluster.xml             |  101 --
 integration-test/config/install-storm.sh        |    1 -
 integration-test/config/install-zookeeper.sh    |    1 +
 integration-test/pom.xml                        |    5 +
 integration-test/run-it.sh                      |    7 +-
 .../topology/window/SlidingTimeCorrectness.java |    4 +-
 .../window/SlidingWindowCorrectness.java        |    2 -
 .../window/TumblingTimeCorrectness.java         |   10 +-
 .../window/TumblingWindowCorrectness.java       |    4 +-
 .../test/java/org/apache/storm/st/DemoTest.java |    3 +-
 .../st/tests/window/SlidingWindowTest.java      |   51 +-
 .../st/tests/window/TumblingWindowTest.java     |    9 +-
 .../apache/storm/st/wrapper/StormCluster.java   |   25 +-
 .../org/apache/storm/st/wrapper/TopoWrap.java   |   87 +-
 pom.xml                                         |   47 +-
 storm-core/pom.xml                              |   37 +-
 .../clj/org/apache/storm/daemon/executor.clj    |   19 +-
 .../clj/org/apache/storm/daemon/logviewer.clj   |   12 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |    3 +-
 .../src/clj/org/apache/storm/ui/helpers.clj     |   21 +-
 storm-core/src/jvm/org/apache/storm/Config.java |    6 +-
 .../apache/storm/blobstore/BlobStoreUtils.java  |    8 +-
 .../storm/blobstore/LocalFsBlobStore.java       |    6 +-
 .../apache/storm/testing/FixedTupleSpout.java   |   18 +-
 .../storm/topology/OutputFieldsGetter.java      |    3 +
 .../storm/topology/WindowedBoltExecutor.java    |    9 +-
 .../spout/IOpaquePartitionedTridentSpout.java   |   22 +-
 .../topology/state/TransactionalState.java      |   14 +-
 .../jvm/org/apache/storm/utils/ConfigUtils.java |   16 +-
 .../apache/storm/windowing/EvictionContext.java |    2 +-
 .../apache/storm/windowing/EvictionPolicy.java  |    2 +-
 .../storm/windowing/TimeEvictionPolicy.java     |    2 +-
 .../storm/windowing/TimeTriggerPolicy.java      |    8 +-
 .../windowing/WaterMarkEventGenerator.java      |   25 +-
 .../windowing/WatermarkCountEvictionPolicy.java |   11 +-
 .../windowing/WatermarkTimeEvictionPolicy.java  |   10 +-
 .../apache/storm/windowing/WindowManager.java   |    3 +
 .../org/apache/storm/integration_test.clj       |   53 +-
 .../clj/org/apache/storm/transactional_test.clj |    4 +-
 .../clj/org/apache/storm/trident/state_test.clj |    4 +-
 .../storm/blobstore/BlobStoreUtilsTest.java     |  135 ++
 .../storm/blobstore/BlobSynchronizerTest.java   |   23 +-
 .../blobstore/MockZookeeperClientBuilder.java   |  100 ++
 .../windowing/WaterMarkEventGeneratorTest.java  |    6 +
 .../storm/windowing/WindowManagerTest.java      |   48 +-
 storm-core/test/resources/log4j2-test.xml       |    2 +
 106 files changed, 5938 insertions(+), 4049 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/storm/blob/99bcf684/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/99bcf684/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------


[36/38] storm git commit: STORM-2153: add taskId to user-defined metrics names

Posted by pt...@apache.org.
STORM-2153: add taskId to user-defined metrics names


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

Branch: refs/heads/1.x-branch
Commit: 1d42d8fb59f5870a671a8cb5c02e1cf46f08a22f
Parents: 7947a07
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Jan 12 14:05:34 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Jan 12 14:05:34 2018 -0500

----------------------------------------------------------------------
 .../src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java   | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/1d42d8fb/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index e0023fd..cfeb711 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -162,6 +162,8 @@ public class StormMetricRegistry {
     }
 
     public static String metricName(String name, TopologyContext context){
+
+
         StringBuilder sb = new StringBuilder("storm.topology.");
         sb.append(context.getStormId());
         sb.append(".");
@@ -169,6 +171,8 @@ public class StormMetricRegistry {
         sb.append(".");
         sb.append(dotToUnderScore(context.getThisComponentId()));
         sb.append(".");
+        sb.append(context.getThisTaskId());
+        sb.append(".");
         sb.append(context.getThisWorkerPort());
         sb.append("-");
         sb.append(name);


[11/38] storm git commit: guard against NPE if a reporter does not have filters

Posted by pt...@apache.org.
guard against NPE if a reporter does not have filters


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

Branch: refs/heads/1.x-branch
Commit: a47b5fbdae0f46eeb4344937b56a6b2126315954
Parents: 20a1cdc
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Sep 29 16:37:25 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Sep 29 16:37:25 2017 -0400

----------------------------------------------------------------------
 .../metrics2/reporters/ScheduledStormReporter.java  | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/a47b5fbd/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
index 6ef39b6..e88b41b 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/reporters/ScheduledStormReporter.java
@@ -74,13 +74,15 @@ public abstract class ScheduledStormReporter implements StormReporter{
     public static StormMetricsFilter getMetricsFilter(Map reporterConf){
         StormMetricsFilter filter = null;
         Map<String, Object> filterConf = (Map)reporterConf.get("filter");
-        String clazz = (String) filterConf.get("class");
-        if(filterConf != null && clazz != null){
-            try {
-                filter = (StormMetricsFilter) Metrics2Utils.instantiate(clazz);
-                filter.prepare(filterConf);
-            } catch (Exception e) {
-                LOG.warn("Unable to instantiate StormMetricsFilter class: {}", clazz);
+        if(filterConf != null) {
+            String clazz = (String) filterConf.get("class");
+            if (clazz != null) {
+                try {
+                    filter = (StormMetricsFilter) Metrics2Utils.instantiate(clazz);
+                    filter.prepare(filterConf);
+                } catch (Exception e) {
+                    LOG.warn("Unable to instantiate StormMetricsFilter class: {}", clazz);
+                }
             }
         }
         return filter;


[37/38] storm git commit: STORM-2153: add taskId to disruptor metrics

Posted by pt...@apache.org.
STORM-2153: add taskId to disruptor metrics


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

Branch: refs/heads/1.x-branch
Commit: 427076ebb6761e80f5ef71bbe6843f21854577c8
Parents: 1d42d8f
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Jan 12 19:50:18 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Jan 12 19:50:18 2018 -0500

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    |  2 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  4 +--
 .../src/clj/org/apache/storm/disruptor.clj      |  4 +--
 .../storm/metrics2/StormMetricRegistry.java     | 26 +++++++++++---------
 .../org/apache/storm/utils/DisruptorQueue.java  |  6 +++--
 .../utils/DisruptorQueueBackpressureTest.java   |  2 +-
 .../apache/storm/utils/DisruptorQueueTest.java  |  4 +--
 7 files changed, 26 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/427076eb/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 3af9b2c..ecbfb14 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -234,7 +234,7 @@
                                   (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
                                   (.getStormId worker-context)
-                                  component-id
+                                  (first task-ids) component-id
                                   (.getThisWorkerPort worker-context)
                                   :producer-type :multi-threaded
                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)

http://git-wip-us.apache.org/repos/asf/storm/blob/427076eb/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index b52da52..dd11959 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -211,7 +211,7 @@
        (map (fn [e] [e (disruptor/disruptor-queue (str "receive-queue" e)
                                                   (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                                  storm-id worker-id port
+                                                  storm-id (int -1) "__system" port
                                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))]))
        (into {})
@@ -256,7 +256,7 @@
         executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
         transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                                  storm-id worker-id port
+                                                  storm-id (int -1) "__system" port
                                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
         executor-receive-queue-map (mk-receive-queue-map storm-conf executors storm-id worker-id port)

http://git-wip-us.apache.org/repos/asf/storm/blob/427076eb/storm-core/src/clj/org/apache/storm/disruptor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/disruptor.clj b/storm-core/src/clj/org/apache/storm/disruptor.clj
index c23c505..6bbf0df 100644
--- a/storm-core/src/clj/org/apache/storm/disruptor.clj
+++ b/storm-core/src/clj/org/apache/storm/disruptor.clj
@@ -28,10 +28,10 @@
    :single-threaded ProducerType/SINGLE})
 
 (defnk disruptor-queue
-  [^String queue-name buffer-size timeout ^String storm-id ^String component-id ^Integer worker-port :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
+  [^String queue-name buffer-size timeout ^String storm-id ^Integer task-id  ^String component-id ^Integer worker-port :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
   (DisruptorQueue. queue-name
                    (PRODUCER-TYPE producer-type) buffer-size
-                   timeout batch-size batch-timeout storm-id component-id worker-port))
+                   timeout batch-size batch-timeout storm-id component-id task-id worker-port))
 
 (defn clojure-handler
   [afn]

http://git-wip-us.apache.org/repos/asf/storm/blob/427076eb/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index cfeb711..1a5bd45 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -46,8 +46,8 @@ public class StormMetricRegistry {
 
     private static String hostName = null;
 
-    public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, String componentId, Integer port){
-        String metricName = metricName(name, topologyId, componentId, port);
+    public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, String componentId, Integer taskId, Integer port){
+        String metricName = metricName(name, topologyId, componentId, taskId, port);
         if(REGISTRY.getGauges().containsKey(metricName)){
             return (SimpleGauge)REGISTRY.getGauges().get(metricName);
         } else {
@@ -55,16 +55,16 @@ public class StormMetricRegistry {
         }
     }
 
-    public static DisruptorMetrics disruptorMetrics(String name, String topologyId, String componentId, Integer port){
+    public static DisruptorMetrics disruptorMetrics(String name, String topologyId, String componentId, Integer taskId, Integer port){
         return new DisruptorMetrics(
-                StormMetricRegistry.gauge(0L, name + "-capacity", topologyId, componentId, port),
-                StormMetricRegistry.gauge(0L, name + "-population", topologyId, componentId, port),
-                StormMetricRegistry.gauge(0L, name + "-write-position", topologyId, componentId, port),
-                StormMetricRegistry.gauge(0L, name + "-read-position", topologyId, componentId, port),
-                StormMetricRegistry.gauge(0.0, name + "-arrival-rate", topologyId, componentId, port),
-                StormMetricRegistry.gauge(0.0, name + "-sojourn-time-ms", topologyId, componentId, port),
-                StormMetricRegistry.gauge(0L, name + "-overflow", topologyId, componentId, port),
-                StormMetricRegistry.gauge(0.0F, name + "-percent-full", topologyId, componentId, port)
+                StormMetricRegistry.gauge(0L, name + "-capacity", topologyId, componentId, taskId, port),
+                StormMetricRegistry.gauge(0L, name + "-population", topologyId, componentId, taskId, port),
+                StormMetricRegistry.gauge(0L, name + "-write-position", topologyId, componentId, taskId, port),
+                StormMetricRegistry.gauge(0L, name + "-read-position", topologyId, componentId, taskId, port),
+                StormMetricRegistry.gauge(0.0, name + "-arrival-rate", topologyId, componentId, taskId, port),
+                StormMetricRegistry.gauge(0.0, name + "-sojourn-time-ms", topologyId, componentId, taskId, port),
+                StormMetricRegistry.gauge(0L, name + "-overflow", topologyId, componentId, taskId, port),
+                StormMetricRegistry.gauge(0.0F, name + "-percent-full", topologyId, componentId, taskId, port)
         );
     }
 
@@ -147,7 +147,7 @@ public class StormMetricRegistry {
         return sb.toString();
     }
 
-    public static String metricName(String name, String stormId, String componentId, Integer workerPort) {
+    public static String metricName(String name, String stormId, String componentId, Integer taskId, Integer workerPort) {
         StringBuilder sb = new StringBuilder("storm.worker.");
         sb.append(stormId);
         sb.append(".");
@@ -155,6 +155,8 @@ public class StormMetricRegistry {
         sb.append(".");
         sb.append(dotToUnderScore(componentId));
         sb.append(".");
+        sb.append(taskId);
+        sb.append(".");
         sb.append(workerPort);
         sb.append("-");
         sb.append(name);

http://git-wip-us.apache.org/repos/asf/storm/blob/427076eb/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
index d7497d6..afa5158 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
@@ -418,7 +418,9 @@ public class DisruptorQueue implements IStatefulObject {
     private final AtomicLong tuplePopulation = new AtomicLong(0);
     private volatile boolean _throttleOn = false;
 
-    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval, String topologyId, String componentId, int port) {
+    // [^String queue-name buffer-size timeout ^String storm-id ^String component-id ^Integer task-id ^Integer worker-port :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
+
+    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval, String topologyId, String componentId, Integer taskId, int port) {
         this._queueName = PREFIX + queueName;
         WaitStrategy wait;
         if (readTimeout <= 0) {
@@ -432,7 +434,7 @@ public class DisruptorQueue implements IStatefulObject {
         _barrier = _buffer.newBarrier();
         _buffer.addGatingSequences(_consumer);
         _metrics = new QueueMetrics();
-        _disruptorMetrics = StormMetricRegistry.disruptorMetrics(_queueName, topologyId, componentId, port);
+        _disruptorMetrics = StormMetricRegistry.disruptorMetrics(_queueName, topologyId, componentId, taskId, port);
         //The batch size can be no larger than half the full queue size.
         //This is mostly to avoid contention issues.
         _inputBatchSize = Math.max(1, Math.min(inputBatchSize, size/2));

http://git-wip-us.apache.org/repos/asf/storm/blob/427076eb/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
index ba2b507..15eb8c4 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
@@ -105,6 +105,6 @@ public class DisruptorQueueBackpressureTest extends TestCase {
     }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", "test",1000);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", "test",1000, 1000);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/427076eb/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
index 59de55d..65d627c 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
@@ -178,10 +178,10 @@ public class DisruptorQueueTest extends TestCase {
     }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", "test", 1000);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", "test", 1000, 1000);
     }
 
     private static DisruptorQueue createQueue(String name, int batchSize, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, batchSize, 1L, "test", "test", 1000);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, batchSize, 1L, "test", "test", 1000, 1000);
     }
 }


[34/38] storm git commit: STORM-2153: move task-metrics from executor to task to avoid map lookup

Posted by pt...@apache.org.
STORM-2153: move task-metrics from executor to task to avoid map lookup


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

Branch: refs/heads/1.x-branch
Commit: b257ba47aae42d5486901a1252cd9b5c0d9ad70e
Parents: 8e2f7e7
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Fri Jan 12 06:52:22 2018 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Fri Jan 12 06:52:22 2018 +0900

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/executor.clj     | 9 ++++-----
 storm-core/src/clj/org/apache/storm/daemon/task.clj         | 9 +++++----
 .../src/jvm/org/apache/storm/metrics2/TaskMetrics.java      | 8 --------
 3 files changed, 9 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b257ba47/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 3dd7289..3af9b2c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -265,7 +265,6 @@
      :stats (mk-executor-stats <> (sampling-rate storm-conf))
      :interval->task->metric-registry (HashMap.)
      :task->component (:task->component worker)
-     :task-metrics (TaskMetrics/taskMetricsMap (first task-ids) (last task-ids) worker-context component-id)
      :stream->component->grouper (outbound-components worker-context component-id storm-conf)
      :report-error (throttled-report-error-fn <>)
      :report-error-and-die (fn [error]
@@ -443,7 +442,7 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-failed-tuple! (:stats executor-data)  (.getFailed ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-failed-tuple! (:stats executor-data)  (.getFailed ^TaskMetrics (:task-metrics task-data) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
   (let [^ISpout spout (:object task-data)
@@ -452,7 +451,7 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-acked-tuple! (:stats executor-data) (.getAcked ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-acked-tuple! (:stats executor-data) (.getAcked ^TaskMetrics (:task-metrics task-data) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)
@@ -821,7 +820,7 @@
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-acked-tuple! executor-stats
-                                                      (.getAcked ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (.getSourceStreamId tuple))
+                                                      (.getAcked ^TaskMetrics (:task-metrics task-data) (.getSourceStreamId tuple))
                                                       (.getSourceComponent tuple)
                                                       (.getSourceStreamId tuple)
                                                       delta))))
@@ -837,7 +836,7 @@
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-failed-tuple! executor-stats
-                                                       (.getFailed ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (.getSourceStreamId tuple))
+                                                       (.getFailed ^TaskMetrics (:task-metrics task-data) (.getSourceStreamId tuple))
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)
                                                        delta))))

http://git-wip-us.apache.org/repos/asf/storm/blob/b257ba47/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 68af75b..82bd2c5 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -143,9 +143,9 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (stats/emitted-tuple! executor-stats (.getEmitted ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream)
+              (stats/emitted-tuple! executor-stats (.getEmitted ^TaskMetrics (:task-metrics task-data) stream) stream)
               (if out-task-id
-                (stats/transferred-tuples! executor-stats (.getTransferred ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream 1)))
+                (stats/transferred-tuples! executor-stats (.getTransferred ^TaskMetrics (:task-metrics task-data) stream) stream 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
@@ -163,8 +163,8 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (stats/emitted-tuple! executor-stats (.getEmitted ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream)
-               (stats/transferred-tuples! executor-stats (.getTransferred ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream (count out-tasks)))
+               (stats/emitted-tuple! executor-stats (.getEmitted ^TaskMetrics (:task-metrics task-data) stream) stream)
+               (stats/transferred-tuples! executor-stats (.getTransferred ^TaskMetrics (:task-metrics task-data) stream) stream (count out-tasks)))
              out-tasks)))
     ))
 
@@ -175,6 +175,7 @@
     :system-context (system-topology-context (:worker executor-data) executor-data task-id)
     :user-context (user-topology-context (:worker executor-data) executor-data task-id)
     :builtin-metrics (builtin-metrics/make-data (:type executor-data) (:stats executor-data))
+    :task-metrics (TaskMetrics. (:worker-context executor-data) (:component-id executor-data) task-id)
     :tasks-fn (mk-tasks-fn <>)
     :object (get-task-object (.getRawTopology ^TopologyContext (:system-context <>)) (:component-id executor-data))))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b257ba47/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
index 550b176..05c62da 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
@@ -78,12 +78,4 @@ public class TaskMetrics {
         }
         return c;
     }
-
-    public static Map<Integer, TaskMetrics> taskMetricsMap(Integer startTaskId, Integer endTaskId, WorkerTopologyContext context, String componentId){
-        Map<Integer, TaskMetrics> retval = new HashMap<>();
-        for (int i = startTaskId; i < endTaskId + 1; i++) {
-            retval.put(i, new TaskMetrics(context, componentId, i));
-        }
-        return retval;
-    }
 }


[13/38] storm git commit: WIP: replace Meter to Counter

Posted by pt...@apache.org.
WIP: replace Meter to Counter


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

Branch: refs/heads/1.x-branch
Commit: e9a9f507eaeb1022615066a98b7822e829f58e0a
Parents: 99bcf68
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Mon Nov 27 17:50:04 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Mon Nov 27 17:50:04 2017 +0900

----------------------------------------------------------------------
 .../src/clj/org/apache/storm/daemon/executor.clj      | 14 +++++++-------
 storm-core/src/clj/org/apache/storm/daemon/task.clj   |  8 ++++----
 .../apache/storm/metrics2/StormMetricRegistry.java    |  6 ++++++
 3 files changed, 17 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e9a9f507/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index c6f206e..94bd7af 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -36,7 +36,7 @@
   (:import [org.apache.storm Config Constants])
   (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
   (:import [org.apache.storm.metrics2 StormMetricRegistry])
-  (:import [com.codahale.metrics Meter])
+  (:import [com.codahale.metrics Meter Counter])
   (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
   (:import [java.util.concurrent ConcurrentLinkedQueue])
   (:require [org.apache.storm [thrift :as thrift]
@@ -280,8 +280,8 @@
                                (log-message "Got interrupted excpetion shutting thread down...")
                                ((:suicide-fn <>))))
      :sampler (mk-stats-sampler storm-conf)
-     :failed-meter (StormMetricRegistry/meter "failed" worker-context component-id)
-     :acked-meter (StormMetricRegistry/meter "acked" worker-context component-id)
+     :failed-meter (StormMetricRegistry/counter "failed" worker-context component-id)
+     :acked-meter (StormMetricRegistry/counter "acked" worker-context component-id)
      :spout-throttling-metrics (if (= executor-type :spout)
                                 (builtin-metrics/make-spout-throttling-data)
                                 nil)
@@ -442,7 +442,7 @@
     ;;TODO: need to throttle these when there's lots of failures
     (when debug?
       (log-message "SPOUT Failing " id ": " tuple-info " REASON: " reason " MSG-ID: " msg-id))
-    (.mark failed-meter)
+    (.inc ^Counter failed-meter)
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
@@ -453,7 +453,7 @@
         task-id (:task-id task-data)
         acked-meter (:acked-meter executor-data)]
     (when debug? (log-message "SPOUT Acking message " id " " msg-id))
-    (.mark acked-meter)
+    (.inc ^Counter acked-meter)
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
@@ -823,7 +823,7 @@
                          (let [delta (tuple-time-delta! tuple)]
                            (when debug? 
                              (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
-                           (.mark  ^Meter (:acked-meter (:executor-data task-data)))
+                           (.inc  ^Counter (:acked-meter (:executor-data task-data)))
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-acked-tuple! executor-stats
@@ -839,7 +839,7 @@
                                debug? (= true (storm-conf TOPOLOGY-DEBUG))]
                            (when debug? 
                              (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
-                           (.mark  ^Meter (:failed-meter (:executor-data task-data)))
+                           (.inc  ^Counter (:failed-meter (:executor-data task-data)))
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-failed-tuple! executor-stats

http://git-wip-us.apache.org/repos/asf/storm/blob/e9a9f507/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 2e4df75..7162f7f 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -28,7 +28,7 @@
   (:import [org.apache.storm.generated ShellComponent JavaObject])
   (:import [org.apache.storm.spout ShellSpout])
   (:import [java.util Collection List ArrayList])
-  (:import [com.codahale.metrics Meter])
+  (:import [com.codahale.metrics Meter Counter])
   (:require [org.apache.storm
              [thrift :as thrift]
              [stats :as stats]])
@@ -131,10 +131,10 @@
         user-context (:user-context task-data)
         executor-stats (:stats executor-data)
         debug? (= true (storm-conf TOPOLOGY-DEBUG))
-        ^Meter emitted-meter (StormMetricRegistry/meter "emitted" worker-context component-id)]
+        ^Counter emitted-meter (StormMetricRegistry/counter "emitted" worker-context component-id)]
         
     (fn ([^Integer out-task-id ^String stream ^List values]
-          (.mark emitted-meter)
+          (.inc ^Counter emitted-meter)
           (when debug?
             (log-message "Emitting direct: " out-task-id "; " component-id " " stream " " values))
           (let [target-component (.getComponentId worker-context out-task-id)
@@ -151,7 +151,7 @@
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
-           (.mark emitted-meter)
+           (.inc ^Counter emitted-meter)
            (when debug?
              (log-message "Emitting: " component-id " " stream " " values))
            (let [out-tasks (ArrayList.)]

http://git-wip-us.apache.org/repos/asf/storm/blob/e9a9f507/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 60d4191..912d888 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -17,6 +17,7 @@
  */
 package org.apache.storm.metrics2;
 
+import com.codahale.metrics.Counter;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.MetricRegistry;
 import org.apache.storm.Config;
@@ -72,6 +73,11 @@ public class StormMetricRegistry {
         return REGISTRY.meter(metricName);
     }
 
+    public static Counter counter(String name, WorkerTopologyContext context, String componentId){
+        String metricName = metricName(name, context.getStormId(), componentId, context.getThisWorkerPort());
+        return REGISTRY.counter(metricName);
+    }
+
     public static void start(Map<String, Object> stormConfig, DaemonType type){
         String localHost = "localhost";
         try {


[27/38] storm git commit: STORM-2153: Use StringBuilder instead of String.format for composing metric names

Posted by pt...@apache.org.
STORM-2153: Use StringBuilder instead of String.format for composing metric names


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

Branch: refs/heads/1.x-branch
Commit: 8bf7252ebd36e540515fedd70fb8c2004c1e4364
Parents: 44cd8ac
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Jan 10 13:43:33 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Jan 10 13:43:33 2018 -0500

----------------------------------------------------------------------
 .../storm/metrics2/StormMetricRegistry.java     | 60 +++++++++++++-------
 1 file changed, 38 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/8bf7252e/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 789367b..aea4539 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -118,7 +118,6 @@ public class StormMetricRegistry {
 
     }
 
-
     public static void stop(){
         for(StormReporter sr : REPORTERS){
             sr.stop();
@@ -126,32 +125,49 @@ public class StormMetricRegistry {
     }
 
     public static String metricName(String name, String stormId, String componentId, String streamId, String executorId, Integer workerPort){
-        return String.format("storm.worker.%s.%s.%s.%s.%s.%s-%s",
-                stormId,
-                hostName,
-                dotToUnderScore(componentId),
-                dotToUnderScore(streamId),
-                dotToUnderScore(executorId),
-                workerPort,
-                name);
+        StringBuilder sb = new StringBuilder("storm.worker.");
+        sb.append(stormId);
+        sb.append(".");
+        sb.append(hostName);
+        sb.append(".");
+        sb.append(dotToUnderScore(componentId));
+        sb.append(".");
+        sb.append(dotToUnderScore(streamId));
+        sb.append(".");
+        sb.append(dotToUnderScore(executorId));
+        sb.append(".");
+        sb.append(workerPort);
+        sb.append("-");
+        sb.append(name);
+        return sb.toString();
     }
 
-    public static String metricName(String name, String stormId, String componentId, Integer workerPort){
-        return String.format("storm.worker.%s.%s.%s.%s-%s",
-                stormId,
-                hostName,
-                dotToUnderScore(componentId),
-                workerPort,
-                name);
+    public static String metricName(String name, String stormId, String componentId, Integer workerPort) {
+        StringBuilder sb = new StringBuilder("storm.worker.");
+        sb.append(stormId);
+        sb.append(".");
+        sb.append(hostName);
+        sb.append(".");
+        sb.append(dotToUnderScore(componentId));
+        sb.append(".");
+        sb.append(workerPort);
+        sb.append("-");
+        sb.append(name);
+        return sb.toString();
     }
 
     public static String metricName(String name, TopologyContext context){
-        return String.format("storm.topology.%s.%s.%s.%s.%s-%s",
-                context.getStormId(),
-                hostName,
-                dotToUnderScore(context.getThisComponentId()),
-                context.getThisWorkerPort(),
-                name);
+        StringBuilder sb = new StringBuilder("storm.topology.");
+        sb.append(context.getStormId());
+        sb.append(".");
+        sb.append(hostName);
+        sb.append(".");
+        sb.append(dotToUnderScore(context.getThisComponentId()));
+        sb.append(".");
+        sb.append(context.getThisWorkerPort());
+        sb.append("-");
+        sb.append(name);
+        return sb.toString();
     }
 
     private static String dotToUnderScore(String str){


[30/38] storm git commit: STORM-2153: add missing type hint

Posted by pt...@apache.org.
STORM-2153: add missing type hint


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

Branch: refs/heads/1.x-branch
Commit: fea11b26780dfd781f7cdb530e8ae6508afc79b4
Parents: d3c00ee
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Jan 11 15:45:42 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Jan 11 15:45:42 2018 -0500

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/task.clj | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/fea11b26/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 26ce76c..68af75b 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -163,7 +163,7 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (stats/emitted-tuple! executor-stats (.getEmitted (.get ^Map (:task-metrics executor-data) task-id) stream) stream)
+               (stats/emitted-tuple! executor-stats (.getEmitted ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream)
                (stats/transferred-tuples! executor-stats (.getTransferred ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream (count out-tasks)))
              out-tasks)))
     ))


[29/38] storm git commit: STORM-2153: eliminate string concatenation when looking up metrics

Posted by pt...@apache.org.
STORM-2153: eliminate string concatenation when looking up metrics


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

Branch: refs/heads/1.x-branch
Commit: d3c00ee7705b2d7b1bba4afd1146fb4c258a471d
Parents: 8d53800
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Jan 11 15:39:37 2018 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Jan 11 15:39:37 2018 -0500

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    | 11 +--
 .../src/clj/org/apache/storm/daemon/task.clj    | 12 ++--
 .../storm/metrics2/StormMetricRegistry.java     |  5 ++
 .../org/apache/storm/metrics2/TaskMetrics.java  | 72 ++++++++++++++++++++
 4 files changed, 89 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d3c00ee7/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index e8d23e5..3dd7289 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -35,7 +35,7 @@
   (:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
   (:import [org.apache.storm Config Constants])
   (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
-  (:import [org.apache.storm.metrics2 StormMetricRegistry])
+  (:import [org.apache.storm.metrics2 StormMetricRegistry TaskMetrics])
   (:import [com.codahale.metrics Meter Counter])
   (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
   (:import [java.util.concurrent ConcurrentLinkedQueue])
@@ -265,6 +265,7 @@
      :stats (mk-executor-stats <> (sampling-rate storm-conf))
      :interval->task->metric-registry (HashMap.)
      :task->component (:task->component worker)
+     :task-metrics (TaskMetrics/taskMetricsMap (first task-ids) (last task-ids) worker-context component-id)
      :stream->component->grouper (outbound-components worker-context component-id storm-conf)
      :report-error (throttled-report-error-fn <>)
      :report-error-and-die (fn [error]
@@ -442,7 +443,7 @@
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-failed-tuple! (:stats executor-data) (StormMetricRegistry/counter "failed" (:worker-context executor-data) (:component-id executor-data) task-id (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-failed-tuple! (:stats executor-data)  (.getFailed ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
   (let [^ISpout spout (:object task-data)
@@ -451,7 +452,7 @@
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
-      (stats/spout-acked-tuple! (:stats executor-data) (StormMetricRegistry/counter "acked" (:worker-context executor-data) (:component-id executor-data) task-id (:stream tuple-info)) (:stream tuple-info) time-delta))))
+      (stats/spout-acked-tuple! (:stats executor-data) (.getAcked ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (:stream tuple-info)) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
   (let [task-ids (:task-ids executor-data)
@@ -820,7 +821,7 @@
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-acked-tuple! executor-stats
-                                                      (StormMetricRegistry/counter "acked" worker-context  (:component-id executor-data) task-id (.getSourceStreamId tuple))
+                                                      (.getAcked ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (.getSourceStreamId tuple))
                                                       (.getSourceComponent tuple)
                                                       (.getSourceStreamId tuple)
                                                       delta))))
@@ -836,7 +837,7 @@
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when (<= 0 delta)
                              (stats/bolt-failed-tuple! executor-stats
-                                                       (StormMetricRegistry/counter "failed" worker-context (:component-id executor-data) task-id (.getSourceStreamId tuple))
+                                                       (.getFailed ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) (.getSourceStreamId tuple))
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)
                                                        delta))))

http://git-wip-us.apache.org/repos/asf/storm/blob/d3c00ee7/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 9e18331..26ce76c 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -23,11 +23,11 @@
   (:import [org.apache.storm.hooks.info SpoutAckInfo SpoutFailInfo
             EmitInfo BoltFailInfo BoltAckInfo])
   (:import [org.apache.storm.task TopologyContext ShellBolt WorkerTopologyContext])
-  (:import [org.apache.storm.metrics2 StormMetricRegistry])
+  (:import [org.apache.storm.metrics2 StormMetricRegistry TaskMetrics])
   (:import [org.apache.storm.utils Utils])
   (:import [org.apache.storm.generated ShellComponent JavaObject])
   (:import [org.apache.storm.spout ShellSpout])
-  (:import [java.util Collection List ArrayList])
+  (:import [java.util Collection List ArrayList Map])
   (:import [com.codahale.metrics Meter Counter])
   (:require [org.apache.storm
              [thrift :as thrift]
@@ -143,9 +143,9 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
-              (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id task-id stream) stream)
+              (stats/emitted-tuple! executor-stats (.getEmitted ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream)
               (if out-task-id
-                (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id task-id stream) stream 1)))
+                (stats/transferred-tuples! executor-stats (.getTransferred ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
@@ -163,8 +163,8 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
-               (stats/emitted-tuple! executor-stats (StormMetricRegistry/counter "emitted" worker-context component-id task-id stream) stream)
-               (stats/transferred-tuples! executor-stats (StormMetricRegistry/counter "transferred" worker-context component-id task-id stream) stream (count out-tasks)))
+               (stats/emitted-tuple! executor-stats (.getEmitted (.get ^Map (:task-metrics executor-data) task-id) stream) stream)
+               (stats/transferred-tuples! executor-stats (.getTransferred ^TaskMetrics (.get ^Map (:task-metrics executor-data) task-id) stream) stream (count out-tasks)))
              out-tasks)))
     ))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/d3c00ee7/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index e1305f9..e0023fd 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -78,6 +78,11 @@ public class StormMetricRegistry {
         return REGISTRY.counter(metricName);
     }
 
+    public static Counter counter(String name, String topologyId, String componentId, Integer taskId, Integer workerPort, String streamId){
+        String metricName = metricName(name, topologyId, componentId, streamId,taskId, workerPort);
+        return REGISTRY.counter(metricName);
+    }
+
     public static void start(Map<String, Object> stormConfig, DaemonType type){
         try {
             hostName = dotToUnderScore(Utils.localHostname());

http://git-wip-us.apache.org/repos/asf/storm/blob/d3c00ee7/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
new file mode 100644
index 0000000..5bb01d2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/TaskMetrics.java
@@ -0,0 +1,72 @@
+package org.apache.storm.metrics2;
+
+import com.codahale.metrics.Counter;
+import org.apache.storm.task.WorkerTopologyContext;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+public class TaskMetrics {
+    ConcurrentMap<String, Counter> ackedByStream = new ConcurrentHashMap<>();
+    ConcurrentMap<String, Counter> failedByStream = new ConcurrentHashMap<>();
+    ConcurrentMap<String, Counter> emittedByStream = new ConcurrentHashMap<>();
+    ConcurrentMap<String, Counter> transferredByStream = new ConcurrentHashMap<>();
+
+    private String topologyId;
+    private String componentId;
+    private Integer taskId;
+    private Integer workerPort;
+
+    public TaskMetrics(WorkerTopologyContext context, String componentId, Integer taskid){
+        this.topologyId = context.getStormId();
+        this.componentId = componentId;
+        this.taskId = taskid;
+        this.workerPort = context.getThisWorkerPort();
+    }
+
+    public Counter getAcked(String streamId) {
+        Counter c = this.ackedByStream.get(streamId);
+        if (c == null) {
+            c = StormMetricRegistry.counter("acked", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            this.ackedByStream.put(streamId, c);
+        }
+        return c;
+    }
+
+    public Counter getFailed(String streamId) {
+        Counter c = this.ackedByStream.get(streamId);
+        if (c == null) {
+            c = StormMetricRegistry.counter("failed", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            this.failedByStream.put(streamId, c);
+        }
+        return c;
+    }
+
+    public Counter getEmitted(String streamId) {
+        Counter c = this.emittedByStream.get(streamId);
+        if (c == null) {
+            c = StormMetricRegistry.counter("emitted", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            this.emittedByStream.put(streamId, c);
+        }
+        return c;
+    }
+
+    public Counter getTransferred(String streamId) {
+        Counter c = this.transferredByStream.get(streamId);
+        if (c == null) {
+            c = StormMetricRegistry.counter("transferred", this.topologyId, this.componentId, this.taskId, this.workerPort, streamId);
+            this.transferredByStream.put(streamId, c);
+        }
+        return c;
+    }
+
+    public static Map<Integer, TaskMetrics> taskMetricsMap(Integer startTaskId, Integer endTaskId, WorkerTopologyContext context, String componentId){
+        Map<Integer, TaskMetrics> retval = new HashMap<>();
+        for (int i = startTaskId; i < endTaskId + 1; i++) {
+            retval.put(i, new TaskMetrics(context, componentId, i));
+        }
+        return retval;
+    }
+}


[10/38] storm git commit: add basic sanity check for metrics v2 config

Posted by pt...@apache.org.
add basic sanity check for metrics v2 config


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

Branch: refs/heads/1.x-branch
Commit: 20a1cdc9ba04050ce8a436d1d587de45c6c6d195
Parents: cfa6cd6
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Sep 29 16:06:22 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Sep 29 16:06:22 2017 -0400

----------------------------------------------------------------------
 storm-core/src/jvm/org/apache/storm/Config.java |  2 +-
 .../storm/validation/ConfigValidation.java      | 40 ++++++++++++++++++++
 2 files changed, 41 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/20a1cdc9/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/Config.java b/storm-core/src/jvm/org/apache/storm/Config.java
index b41b2aa..2a41a94 100644
--- a/storm-core/src/jvm/org/apache/storm/Config.java
+++ b/storm-core/src/jvm/org/apache/storm/Config.java
@@ -139,7 +139,7 @@ public class Config extends HashMap<String, Object> {
     @isString
     public static final String STORM_META_SERIALIZATION_DELEGATE = "storm.meta.serialization.delegate";
 
-    @isType(type=List.class)
+    @isListEntryCustom(entryValidatorClasses={MetricReportersValidator.class})
     public static final String STORM_METRICS_REPORTERS = "storm.metrics.reporters";
 
     /**

http://git-wip-us.apache.org/repos/asf/storm/blob/20a1cdc9/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
index fb220dd..8bcea84 100644
--- a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
@@ -493,6 +493,46 @@ public class ConfigValidation {
         }
     }
 
+    public static class MetricReportersValidator extends Validator {
+
+        @Override
+        public void validateField(String name, Object o) {
+            if(o == null) {
+                return;
+            }
+            SimpleTypeValidator.validateField(name, Map.class, o);
+            if(!((Map) o).containsKey("class") ) {
+                throw new IllegalArgumentException( "Field " + name + " must have map entry with key: class");
+            }
+            if(!((Map) o).containsKey("daemons") ) {
+                throw new IllegalArgumentException("Field " + name + " must have map entry with key: daemons");
+            } else {
+                // daemons can only be 'nimbus', 'supervisor', or 'worker'
+                Object list = ((Map)o).get("daemons");
+                if(list == null || !(list instanceof List)){
+                    throw new IllegalArgumentException("Field 'daemons' must be a non-null list.");
+                }
+                List daemonList = (List)list;
+                for(Object string : daemonList){
+                    if (string instanceof String &&
+                            (((String) string).equals("nimbus") ||
+                                    ((String) string).equals("supervisor") ||
+                                    ((String) string).equals("worker"))) {
+                        return;
+                    }
+                    throw new IllegalArgumentException("Field daemons must contain at least one of \"nimbus\", \"supervisor\", or \"worker\"");
+                }
+
+            }
+            if(((Map)o).containsKey("filter")){
+                Map filterMap = (Map)((Map)o).get("filter");
+                SimpleTypeValidator.validateField("filter", String.class, filterMap.get("class"));
+            }
+            SimpleTypeValidator.validateField(name, String.class, ((Map) o).get("class"));
+
+        }
+    }
+
     public static class MapOfStringToMapOfStringToObjectValidator extends Validator {
       @Override
       public  void validateField(String name, Object o) {


[08/38] storm git commit: remove unnecessary TODO comment

Posted by pt...@apache.org.
remove unnecessary TODO comment


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

Branch: refs/heads/1.x-branch
Commit: de20cbd8f7ee588e5d9d2e6d04ceeee43af4fc12
Parents: a169356
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Sep 5 16:12:24 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Sep 5 16:12:24 2017 -0400

----------------------------------------------------------------------
 storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/de20cbd8/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java b/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
index 994a965..22eb6c5 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/DisruptorMetrics.java
@@ -24,7 +24,7 @@ public class DisruptorMetrics {
     private SimpleGauge<Long> population;
     private SimpleGauge<Long> writePosition;
     private SimpleGauge<Long> readPosition;
-    private SimpleGauge<Double> arrivalRate; // TODO: Change to meter
+    private SimpleGauge<Double> arrivalRate;
     private SimpleGauge<Double> sojournTime;
     private SimpleGauge<Long> overflow;
     private SimpleGauge<Float> pctFull;


[22/38] storm git commit: Merge branch '1.x-branch' into metrics_v2

Posted by pt...@apache.org.
Merge branch '1.x-branch' into metrics_v2


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

Branch: refs/heads/1.x-branch
Commit: b5ae9c3426e69eba11446d055649307daecb05c7
Parents: 47be75c b8f76af
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Dec 20 15:28:37 2017 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Dec 20 15:28:37 2017 -0500

----------------------------------------------------------------------
 .travis.yml                                     |   1 +
 conf/storm.yaml.example                         |   9 +-
 docs/Eventlogging.md                            |  33 ++-
 .../flux/examples/StatefulWordCounter.java      |   7 +-
 .../storm/flux/examples/TestPrintBolt.java      |   3 +-
 .../storm/flux/examples/TestWindowBolt.java     |   5 +-
 .../storm/flux/examples/WordCountClient.java    |  20 +-
 .../apache/storm/flux/examples/WordCounter.java |  11 +-
 .../main/java/org/apache/storm/flux/Flux.java   |  69 +++---
 .../java/org/apache/storm/flux/FluxBuilder.java | 234 ++++++++++++-------
 .../apache/storm/flux/api/TopologySource.java   |   8 +-
 .../org/apache/storm/flux/model/BeanDef.java    |   1 +
 .../storm/flux/model/BeanListReference.java     |   5 +-
 .../apache/storm/flux/model/BeanReference.java  |   5 +-
 .../org/apache/storm/flux/model/BoltDef.java    |   1 +
 .../storm/flux/model/ConfigMethodDef.java       |  13 +-
 .../storm/flux/model/ExecutionContext.java      |  23 +-
 .../apache/storm/flux/model/GroupingDef.java    |   3 +-
 .../org/apache/storm/flux/model/IncludeDef.java |   3 +-
 .../org/apache/storm/flux/model/ObjectDef.java  |  64 ++++-
 .../apache/storm/flux/model/PropertyDef.java    |  17 +-
 .../org/apache/storm/flux/model/SpoutDef.java   |   1 +
 .../org/apache/storm/flux/model/StreamDef.java  |   3 +-
 .../apache/storm/flux/model/TopologyDef.java    | 114 ++++++---
 .../storm/flux/model/TopologySourceDef.java     |   3 +-
 .../org/apache/storm/flux/model/VertexDef.java  |   3 +-
 .../apache/storm/flux/parser/FluxParser.java    | 157 ++++++++-----
 .../java/org/apache/storm/flux/TCKTest.java     |   4 +
 .../org/apache/storm/flux/test/TestBolt.java    |  21 +-
 .../resources/configs/config-methods-test.yaml  |  38 +++
 .../flux/wrappers/bolts/FluxShellBolt.java      |  38 +--
 .../storm/flux/wrappers/bolts/LogInfoBolt.java  |   2 +-
 .../flux/wrappers/spouts/FluxShellSpout.java    |  36 +--
 .../apache/storm/kafka/spout/KafkaSpout.java    |  84 ++++---
 .../storm/kafka/spout/KafkaSpoutConfig.java     |  98 ++++----
 .../spout/ManualPartitionSubscription.java      |   2 +-
 .../storm/kafka/spout/KafkaSpoutConfigTest.java | 150 +++++++++++-
 .../storm/kafka/spout/KafkaSpoutEmitTest.java   |  40 +++-
 .../kafka/spout/MaxUncommittedOffsetTest.java   |  53 ++++-
 .../ManualPartitionSubscriptionTest.java        |  79 +++++++
 storm-core/src/jvm/org/apache/storm/Config.java |  40 +++-
 .../apache/storm/messaging/netty/Client.java    |   7 +-
 .../apache/storm/metric/EventLoggerBolt.java    |  61 ++++-
 .../storm/metric/FileBasedEventLogger.java      |  11 +-
 .../org/apache/storm/metric/IEventLogger.java   |  41 +++-
 .../storm/validation/ConfigValidation.java      |  20 ++
 46 files changed, 1211 insertions(+), 430 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b5ae9c34/storm-core/src/jvm/org/apache/storm/Config.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/b5ae9c34/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
index 89aecee,0602dbf..d7ca48d
--- a/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
+++ b/storm-core/src/jvm/org/apache/storm/validation/ConfigValidation.java
@@@ -491,49 -491,28 +491,69 @@@ public class ConfigValidation 
              SimpleTypeValidator.validateField(name, String.class, ((Map) o).get("class"));
              new IntegerValidator().validateField(name, ((Map) o).get("parallelism.hint"));
          }
 +    }
 +
 +    public static class MetricReportersValidator extends Validator {
 +
 +        @Override
 +        public void validateField(String name, Object o) {
 +            if(o == null) {
 +                return;
 +            }
 +            SimpleTypeValidator.validateField(name, Map.class, o);
 +            if(!((Map) o).containsKey("class") ) {
 +                throw new IllegalArgumentException( "Field " + name + " must have map entry with key: class");
 +            }
 +            if(!((Map) o).containsKey("daemons") ) {
 +                throw new IllegalArgumentException("Field " + name + " must have map entry with key: daemons");
 +            } else {
 +                // daemons can only be 'nimbus', 'supervisor', or 'worker'
 +                Object list = ((Map)o).get("daemons");
 +                if(list == null || !(list instanceof List)){
 +                    throw new IllegalArgumentException("Field 'daemons' must be a non-null list.");
 +                }
 +                List daemonList = (List)list;
 +                for(Object string : daemonList){
 +                    if (string instanceof String &&
 +                            (((String) string).equals("nimbus") ||
 +                                    ((String) string).equals("supervisor") ||
 +                                    ((String) string).equals("worker"))) {
 +                        continue;
 +                    }
 +                    throw new IllegalArgumentException("Field 'daemons' must contain at least one of the following:" +
 +                            " \"nimbus\", \"supervisor\", or \"worker\"");
 +                }
 +
 +            }
 +            if(((Map)o).containsKey("filter")){
 +                Map filterMap = (Map)((Map)o).get("filter");
 +                SimpleTypeValidator.validateField("class", String.class, filterMap.get("class"));
 +            }
 +            SimpleTypeValidator.validateField(name, String.class, ((Map) o).get("class"));
 +
 +        }
      }
  
+     public static class EventLoggerRegistryValidator extends Validator {
+ 
+         @Override
+         public void validateField(String name, Object o) {
+             if(o == null) {
+                 return;
+             }
+             SimpleTypeValidator.validateField(name, Map.class, o);
+             if(!((Map<?, ?>) o).containsKey("class") ) {
+                 throw new IllegalArgumentException( "Field " + name + " must have map entry with key: class");
+             }
+ 
+             SimpleTypeValidator.validateField(name, String.class, ((Map<?, ?>) o).get("class"));
+ 
+             if(((Map<?, ?>) o).containsKey("arguments") ) {
+                 SimpleTypeValidator.validateField(name, Map.class, ((Map<?, ?>) o).get("arguments"));
+             }
+         }
+     }
+ 
      public static class MapOfStringToMapOfStringToObjectValidator extends Validator {
        @Override
        public  void validateField(String name, Object o) {


[04/38] storm git commit: expand disruptor metrics name to include component/worker id

Posted by pt...@apache.org.
expand disruptor metrics name to include component/worker id


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

Branch: refs/heads/1.x-branch
Commit: 6eaa1a85a340801d066192c118ba87281b2f2313
Parents: 0be278a
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Fri Aug 11 16:08:34 2017 -0400
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Fri Aug 11 16:08:34 2017 -0400

----------------------------------------------------------------------
 .../clj/org/apache/storm/daemon/executor.clj    |  1 +
 .../src/clj/org/apache/storm/daemon/worker.clj  |  8 +++----
 .../src/clj/org/apache/storm/disruptor.clj      |  4 ++--
 .../storm/metrics2/StormMetricRegistry.java     | 22 ++++++++++----------
 .../org/apache/storm/utils/DisruptorQueue.java  |  4 ++--
 .../utils/DisruptorQueueBackpressureTest.java   |  2 +-
 .../apache/storm/utils/DisruptorQueueTest.java  |  4 ++--
 7 files changed, 23 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/6eaa1a85/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 3e5dd20..95e43f6 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -234,6 +234,7 @@
                                   (storm-conf TOPOLOGY-EXECUTOR-SEND-BUFFER-SIZE)
                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
                                   (.getStormId worker-context)
+                                  component-id
                                   (.getThisWorkerPort worker-context)
                                   :producer-type :single-threaded
                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)

http://git-wip-us.apache.org/repos/asf/storm/blob/6eaa1a85/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index b2810db..b52da52 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -205,13 +205,13 @@
           (transfer-fn serializer tuple-batch)))
       transfer-fn)))
 
-(defn- mk-receive-queue-map [storm-conf executors storm-id port]
+(defn- mk-receive-queue-map [storm-conf executors storm-id worker-id port]
   (->> executors
        ;; TODO: this depends on the type of executor
        (map (fn [e] [e (disruptor/disruptor-queue (str "receive-queue" e)
                                                   (storm-conf TOPOLOGY-EXECUTOR-RECEIVE-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                                  storm-id port
+                                                  storm-id worker-id port
                                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))]))
        (into {})
@@ -256,10 +256,10 @@
         executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
         transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
                                                   (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                                  storm-id port
+                                                  storm-id worker-id port
                                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
                                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
-        executor-receive-queue-map (mk-receive-queue-map storm-conf executors storm-id port)
+        executor-receive-queue-map (mk-receive-queue-map storm-conf executors storm-id worker-id port)
 
         receive-queue-map (->> executor-receive-queue-map
                                (mapcat (fn [[e queue]] (for [t (executor-id->tasks e)] [t queue])))

http://git-wip-us.apache.org/repos/asf/storm/blob/6eaa1a85/storm-core/src/clj/org/apache/storm/disruptor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/disruptor.clj b/storm-core/src/clj/org/apache/storm/disruptor.clj
index 73a9d84..c23c505 100644
--- a/storm-core/src/clj/org/apache/storm/disruptor.clj
+++ b/storm-core/src/clj/org/apache/storm/disruptor.clj
@@ -28,10 +28,10 @@
    :single-threaded ProducerType/SINGLE})
 
 (defnk disruptor-queue
-  [^String queue-name buffer-size timeout ^String storm-id ^Integer worker-port :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
+  [^String queue-name buffer-size timeout ^String storm-id ^String component-id ^Integer worker-port :producer-type :multi-threaded :batch-size 100 :batch-timeout 1]
   (DisruptorQueue. queue-name
                    (PRODUCER-TYPE producer-type) buffer-size
-                   timeout batch-size batch-timeout storm-id worker-port))
+                   timeout batch-size batch-timeout storm-id component-id worker-port))
 
 (defn clojure-handler
   [afn]

http://git-wip-us.apache.org/repos/asf/storm/blob/6eaa1a85/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
index 845745f..4c975a3 100644
--- a/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
+++ b/storm-core/src/jvm/org/apache/storm/metrics2/StormMetricRegistry.java
@@ -43,9 +43,9 @@ public class StormMetricRegistry {
 
     private static String hostName = null;
 
-    public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, Integer port){
+    public static <T> SimpleGauge<T>  gauge(T initialValue, String name, String topologyId, String componentId, Integer port){
         SimpleGauge<T> gauge = new SimpleGauge<>(initialValue);
-        String metricName = metricName(name, topologyId, null, port);
+        String metricName = metricName(name, topologyId, componentId, port);
             if(REGISTRY.getGauges().containsKey(metricName)){
                 return (SimpleGauge)REGISTRY.getGauges().get(metricName);
         } else {
@@ -53,16 +53,16 @@ public class StormMetricRegistry {
         }
     }
 
-    public static DisruptorMetrics disruptorMetrics(String name, String topologyId, Integer port){
+    public static DisruptorMetrics disruptorMetrics(String name, String topologyId, String componentId, Integer port){
         return new DisruptorMetrics(
-                StormMetricRegistry.gauge(0L, name + "-capacity", topologyId, port),
-                StormMetricRegistry.gauge(0L, name + "-population", topologyId, port),
-                StormMetricRegistry.gauge(0L, name + "-write-position", topologyId, port),
-                StormMetricRegistry.gauge(0L, name + "-read-position", topologyId, port),
-                StormMetricRegistry.gauge(0.0, name + "-arrival-rate", topologyId, port),
-                StormMetricRegistry.gauge(0.0, name + "-sojourn-time-ms", topologyId, port),
-                StormMetricRegistry.gauge(0L, name + "-overflow", topologyId, port),
-                StormMetricRegistry.gauge(0.0F, name + "-percent-full", topologyId, port)
+                StormMetricRegistry.gauge(0L, name + "-capacity", topologyId, componentId, port),
+                StormMetricRegistry.gauge(0L, name + "-population", topologyId, componentId, port),
+                StormMetricRegistry.gauge(0L, name + "-write-position", topologyId, componentId, port),
+                StormMetricRegistry.gauge(0L, name + "-read-position", topologyId, componentId, port),
+                StormMetricRegistry.gauge(0.0, name + "-arrival-rate", topologyId, componentId, port),
+                StormMetricRegistry.gauge(0.0, name + "-sojourn-time-ms", topologyId, componentId, port),
+                StormMetricRegistry.gauge(0L, name + "-overflow", topologyId, componentId, port),
+                StormMetricRegistry.gauge(0.0F, name + "-percent-full", topologyId, componentId, port)
         );
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/6eaa1a85/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
index 35bc83f..5c0a2fb 100644
--- a/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
+++ b/storm-core/src/jvm/org/apache/storm/utils/DisruptorQueue.java
@@ -404,7 +404,7 @@ public class DisruptorQueue implements IStatefulObject {
     private final AtomicLong _overflowCount = new AtomicLong(0);
     private volatile boolean _throttleOn = false;
 
-    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval, String topologyId, int port) {
+    public DisruptorQueue(String queueName, ProducerType type, int size, long readTimeout, int inputBatchSize, long flushInterval, String topologyId, String componentId, int port) {
         this._queueName = PREFIX + queueName;
         WaitStrategy wait;
         if (readTimeout <= 0) {
@@ -418,7 +418,7 @@ public class DisruptorQueue implements IStatefulObject {
         _barrier = _buffer.newBarrier();
         _buffer.addGatingSequences(_consumer);
         _metrics = new QueueMetrics();
-        _disruptorMetrics = StormMetricRegistry.disruptorMetrics(_queueName, topologyId, port);
+        _disruptorMetrics = StormMetricRegistry.disruptorMetrics(_queueName, topologyId, componentId, port);
         //The batch size can be no larger than half the full queue size.
         //This is mostly to avoid contention issues.
         _inputBatchSize = Math.max(1, Math.min(inputBatchSize, size/2));

http://git-wip-us.apache.org/repos/asf/storm/blob/6eaa1a85/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
index 110fe88..ba2b507 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueBackpressureTest.java
@@ -105,6 +105,6 @@ public class DisruptorQueueBackpressureTest extends TestCase {
     }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", 1000);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", "test",1000);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/6eaa1a85/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
index c834cbb..59de55d 100644
--- a/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
+++ b/storm-core/test/jvm/org/apache/storm/utils/DisruptorQueueTest.java
@@ -178,10 +178,10 @@ public class DisruptorQueueTest extends TestCase {
     }
 
     private static DisruptorQueue createQueue(String name, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", 1000);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, 1, 1L, "test", "test", 1000);
     }
 
     private static DisruptorQueue createQueue(String name, int batchSize, int queueSize) {
-        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, batchSize, 1L, "test", 1000);
+        return new DisruptorQueue(name, ProducerType.MULTI, queueSize, 0L, batchSize, 1L, "test", "test", 1000);
     }
 }


[14/38] storm git commit: WIP apply sampling to new metrics

Posted by pt...@apache.org.
WIP apply sampling to new metrics


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

Branch: refs/heads/1.x-branch
Commit: 85dbacdd058ee8b3246ff6982a4079713923b66e
Parents: e9a9f50
Author: Jungtaek Lim <ka...@gmail.com>
Authored: Tue Nov 28 11:51:45 2017 +0900
Committer: Jungtaek Lim <ka...@gmail.com>
Committed: Tue Nov 28 11:51:45 2017 +0900

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/executor.clj | 8 ++++----
 storm-core/src/clj/org/apache/storm/daemon/task.clj     | 4 ++--
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/85dbacdd/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 94bd7af..720bfa7 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -442,10 +442,10 @@
     ;;TODO: need to throttle these when there's lots of failures
     (when debug?
       (log-message "SPOUT Failing " id ": " tuple-info " REASON: " reason " MSG-ID: " msg-id))
-    (.inc ^Counter failed-meter)
     (.fail spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutFail (SpoutFailInfo. msg-id task-id time-delta))
     (when time-delta
+      (.inc ^Counter failed-meter)
       (stats/spout-failed-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn- ack-spout-msg [executor-data task-data msg-id tuple-info time-delta id debug?]
@@ -453,10 +453,10 @@
         task-id (:task-id task-data)
         acked-meter (:acked-meter executor-data)]
     (when debug? (log-message "SPOUT Acking message " id " " msg-id))
-    (.inc ^Counter acked-meter)
     (.ack spout msg-id)
     (task/apply-hooks (:user-context task-data) .spoutAck (SpoutAckInfo. msg-id task-id time-delta))
     (when time-delta
+      (.inc ^Counter acked-meter)
       (stats/spout-acked-tuple! (:stats executor-data) (:stream tuple-info) time-delta))))
 
 (defn mk-task-receiver [executor-data tuple-action-fn]
@@ -823,9 +823,9 @@
                          (let [delta (tuple-time-delta! tuple)]
                            (when debug? 
                              (log-message "BOLT ack TASK: " task-id " TIME: " delta " TUPLE: " tuple))
-                           (.inc  ^Counter (:acked-meter (:executor-data task-data)))
                            (task/apply-hooks user-context .boltAck (BoltAckInfo. tuple task-id delta))
                            (when (<= 0 delta)
+                             (.inc ^Counter (:acked-meter (:executor-data task-data)))
                              (stats/bolt-acked-tuple! executor-stats
                                                       (.getSourceComponent tuple)
                                                       (.getSourceStreamId tuple)
@@ -839,9 +839,9 @@
                                debug? (= true (storm-conf TOPOLOGY-DEBUG))]
                            (when debug? 
                              (log-message "BOLT fail TASK: " task-id " TIME: " delta " TUPLE: " tuple))
-                           (.inc  ^Counter (:failed-meter (:executor-data task-data)))
                            (task/apply-hooks user-context .boltFail (BoltFailInfo. tuple task-id delta))
                            (when (<= 0 delta)
+                             (.inc  ^Counter (:failed-meter (:executor-data task-data)))
                              (stats/bolt-failed-tuple! executor-stats
                                                        (.getSourceComponent tuple)
                                                        (.getSourceStreamId tuple)

http://git-wip-us.apache.org/repos/asf/storm/blob/85dbacdd/storm-core/src/clj/org/apache/storm/daemon/task.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/task.clj b/storm-core/src/clj/org/apache/storm/daemon/task.clj
index 7162f7f..c43d20d 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/task.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/task.clj
@@ -134,7 +134,6 @@
         ^Counter emitted-meter (StormMetricRegistry/counter "emitted" worker-context component-id)]
         
     (fn ([^Integer out-task-id ^String stream ^List values]
-          (.inc ^Counter emitted-meter)
           (when debug?
             (log-message "Emitting direct: " out-task-id "; " component-id " " stream " " values))
           (let [target-component (.getComponentId worker-context out-task-id)
@@ -145,13 +144,13 @@
               (throw (IllegalArgumentException. "Cannot emitDirect to a task expecting a regular grouping")))                          
             (apply-hooks user-context .emit (EmitInfo. values stream task-id [out-task-id]))
             (when (emit-sampler)
+              (.inc ^Counter emitted-meter)
               (stats/emitted-tuple! executor-stats stream)
               (if out-task-id
                 (stats/transferred-tuples! executor-stats stream 1)))
             (if out-task-id [out-task-id])
             ))
         ([^String stream ^List values]
-           (.inc ^Counter emitted-meter)
            (when debug?
              (log-message "Emitting: " component-id " " stream " " values))
            (let [out-tasks (ArrayList.)]
@@ -166,6 +165,7 @@
                    )))
              (apply-hooks user-context .emit (EmitInfo. values stream task-id out-tasks))
              (when (emit-sampler)
+               (.inc ^Counter emitted-meter)
                (stats/emitted-tuple! executor-stats stream)
                (stats/transferred-tuples! executor-stats stream (count out-tasks)))
              out-tasks)))