You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by revans2 <gi...@git.apache.org> on 2017/07/10 18:26:33 UTC

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

GitHub user revans2 opened a pull request:

    https://github.com/apache/storm/pull/2200

    STORM-2616: Documentation for built in metrics

    I am happy to pull this back into 1.x branches too.  I am not sure exactly how the metrics might have changed though, so we might want to be careful about doing so.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/revans2/incubator-storm STORM-2616

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/2200.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2200
    
----

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126516261
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    --- End diff --
    
    an `__ack-count` metrics -> the `__ack-count` metric


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126518695
  
    --- Diff: docs/Metrics.md ---
    @@ -44,7 +45,7 @@ topology.metrics.consumer.register:
         argument: "http://example.com:8080/metrics/my-topology/"
     ```
     
    -Storm appends MetricsConsumerBolt to your topology per each registered metrics consumer internally, and each MetricsConsumerBolt subscribes to receive metrics from all tasks. The parallelism for that Bolt is set to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_<metrics consumer class name>`. If you register same class name more than once, postfix `#<sequence number>` is appended to component id.
    +Storm appends MetricsConsumerBolt to your topology per each registered metrics consumer internally, and each MetricsConsumerBolt subscribes to receive metrics from all tasks. The parallelism for that Bolt is set to `parallelism.hint` and `component id` for that Bolt is set to `__metrics_<metrics consumer class name>`. If you register the same class name more than once, postfix `#<sequence number>` is appended to component id.
    --- End diff --
    
    Should this say "Storm appends one MetricsConsumerBolt [...] per registered metrics consumer"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126533215
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    +
    +`arrival_rate_secs` is an estimation of the number of tuple that are inserted into the queue in one second, although it is actually the dequeue rate.
    +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each entry sits in the queue before it is processed.
    +
    +A disruptor queue has a set number of slots.  If the regular queue fills up an overflow queue takes over.  The number of tuple batches stored in this overflow section are represented by the `overflow` metric.  Storm also does some micro batching of tuples for performance/efficiency reasons so you may see the overflow with a very small number in it even if the queue is not full.
    +
    +`read_pos` and `write_pos` are internal disruptor accounting numbers.  You can think of them almost as the total number of tuple batches written (`write_pos`) or read (`read_pos`) since the queue was created.  They allow for integer overflow so if you use them please take that into account.
    +
    +`capacity` is the total number of slots in the queue. `population` is the number of slots taken in the queue.
    +
    +#### System Bolt (Worker) Metrics
    +
    +The System Bolt `__system` provides lots of metrics for different worker wide things.  The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above.
    +
    +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it.
    +
    +##### Receive (NettyServer)
    +`__recv-iconnection` under the `__system` bolt reports stats for the netty server on the worker.  This is what gets messages from other workers.  It is of the form
    +
    +```
    +{
    +    "dequeuedMessages": 0,
    +    "enqueued": {
    +      "/127.0.0.1:49952": 389951
    +    }
    +}
    +```
    +
    +`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts.  That is no longer the case and the value can be ignored.
    +`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker.
    +
    +##### Send (Netty Client)
    +
    +The `__send-iconnection` metric of the `__system` bolt holds information about all of the clients for this worker that are sending metrics.  It is of the form
    +
    +```
    +{
    +    NodeInfo(node:7decee4b-c314-41f4-b362-fd1358c985b3-127.0.01, port:[6701]): {
    +        "reconnects": 0,
    +        "src": "/127.0.0.1:49951",
    +        "pending": 0,
    +        "dest": "localhost/127.0.0.1:6701",
    +        "sent": 420779,
    +        "lostOnSend": 0
    +    }
    +}
    +```
    +
    +The value is a map where the key is a NodeInfo class for the downstream worker it is sending messages to.  This is the SupervisorId + port.  The value is another map with the fields
    +
    +`src`  What host/port this client has used to connect to the receiving worker.
    +`dest` What host/port this client has connected to.
    +`reconnects` the number of reconnections that have happened.
    +`pending` the number of messages that have not been sent.  (This corresponds to messages, not tuples)
    +`sent` the number of messages that have been send.  (This is messages not tuples)
    +`lostOnSend`.  This is the number of messages that were lost because of connection issues. (This is messages not tuples). 
    +
    +##### JVM Memory
    +
    +The JVM running the worker has several metrics relating to memory usage.  We pull them out of the JVM and report them here.  The relevant metrics are `memory/nonHeap` for off heap memory and `memory/heap` for on heap memory.  These values come from the [MemoryUsage](https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html) mxbean for details.
    +
    +The fields reported are
    +
    +`maxBytes`	`memUsage.getMax()`
    --- End diff --
    
    This doesn't render well. Maybe use colon to indicate which names and values belong together?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126529945
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    +
    +`arrival_rate_secs` is an estimation of the number of tuple that are inserted into the queue in one second, although it is actually the dequeue rate.
    +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each entry sits in the queue before it is processed.
    +
    +A disruptor queue has a set number of slots.  If the regular queue fills up an overflow queue takes over.  The number of tuple batches stored in this overflow section are represented by the `overflow` metric.  Storm also does some micro batching of tuples for performance/efficiency reasons so you may see the overflow with a very small number in it even if the queue is not full.
    +
    +`read_pos` and `write_pos` are internal disruptor accounting numbers.  You can think of them almost as the total number of tuple batches written (`write_pos`) or read (`read_pos`) since the queue was created.  They allow for integer overflow so if you use them please take that into account.
    +
    +`capacity` is the total number of slots in the queue. `population` is the number of slots taken in the queue.
    +
    +#### System Bolt (Worker) Metrics
    +
    +The System Bolt `__system` provides lots of metrics for different worker wide things.  The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above.
    +
    +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it.
    +
    +##### Receive (NettyServer)
    +`__recv-iconnection` under the `__system` bolt reports stats for the netty server on the worker.  This is what gets messages from other workers.  It is of the form
    +
    +```
    +{
    +    "dequeuedMessages": 0,
    +    "enqueued": {
    +      "/127.0.0.1:49952": 389951
    +    }
    +}
    +```
    +
    +`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts.  That is no longer the case and the value can be ignored.
    +`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker.
    +
    +##### Send (Netty Client)
    +
    +The `__send-iconnection` metric of the `__system` bolt holds information about all of the clients for this worker that are sending metrics.  It is of the form
    +
    +```
    +{
    +    NodeInfo(node:7decee4b-c314-41f4-b362-fd1358c985b3-127.0.01, port:[6701]): {
    +        "reconnects": 0,
    +        "src": "/127.0.0.1:49951",
    +        "pending": 0,
    +        "dest": "localhost/127.0.0.1:6701",
    +        "sent": 420779,
    +        "lostOnSend": 0
    +    }
    +}
    +```
    +
    +The value is a map where the key is a NodeInfo class for the downstream worker it is sending messages to.  This is the SupervisorId + port.  The value is another map with the fields
    +
    +`src`  What host/port this client has used to connect to the receiving worker.
    +`dest` What host/port this client has connected to.
    +`reconnects` the number of reconnections that have happened.
    +`pending` the number of messages that have not been sent.  (This corresponds to messages, not tuples)
    +`sent` the number of messages that have been send.  (This is messages not tuples)
    +`lostOnSend`.  This is the number of messages that were lost because of connection issues. (This is messages not tuples). 
    +
    +##### JVM Memory
    +
    +The JVM running the worker has several metrics relating to memory usage.  We pull them out of the JVM and report them here.  The relevant metrics are `memory/nonHeap` for off heap memory and `memory/heap` for on heap memory.  These values come from the [MemoryUsage](https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html) mxbean for details.
    --- End diff --
    
    "for details" doesn't fit the rest of the sentence. Also nit: "report them here" -> "report them as metrics"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126515845
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    --- End diff --
    
    later -> latter


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126514663
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    --- End diff --
    
    I'm a little unsure what you're saying here. Is it possible to change the bucket size dynamically at runtime, or are you saying that some consumers will break if the topology isn't using the default bucket size?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126519199
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    --- End diff --
    
    But the spout might instead show something like the following for the same metric


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on the issue:

    https://github.com/apache/storm/pull/2200
  
    Thanks, it looks great. +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126551738
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    --- End diff --
    
    Nevermind, I just googled it and the first result is for this meaning. I was just not aware :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126519643
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    --- End diff --
    
    Same here. Also "It could be because" would probably be more precise as "Tuple trees may fail because"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126548655
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    --- End diff --
    
    I don't think we introduce 'critical path' anywhere in Storm but we already know about that. Isn't it widely-used word? If there's more widely-used representation we may be better to change.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126522426
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    --- End diff --
    
    "The longer this gets the fewer tuples a single bolt instance can process" -> "The higher this gets, the lower the throughput of tuples per bolt instance"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126526874
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    +
    +`arrival_rate_secs` is an estimation of the number of tuple that are inserted into the queue in one second, although it is actually the dequeue rate.
    --- End diff --
    
    tuple -> tuples.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2200: STORM-2616: Documentation for built in metrics

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the issue:

    https://github.com/apache/storm/pull/2200
  
    @srdo I think I have addressed all of your concerns.  Great suggestions by the way.  Please have another look.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126529604
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    +
    +`arrival_rate_secs` is an estimation of the number of tuple that are inserted into the queue in one second, although it is actually the dequeue rate.
    +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each entry sits in the queue before it is processed.
    +
    +A disruptor queue has a set number of slots.  If the regular queue fills up an overflow queue takes over.  The number of tuple batches stored in this overflow section are represented by the `overflow` metric.  Storm also does some micro batching of tuples for performance/efficiency reasons so you may see the overflow with a very small number in it even if the queue is not full.
    +
    +`read_pos` and `write_pos` are internal disruptor accounting numbers.  You can think of them almost as the total number of tuple batches written (`write_pos`) or read (`read_pos`) since the queue was created.  They allow for integer overflow so if you use them please take that into account.
    +
    +`capacity` is the total number of slots in the queue. `population` is the number of slots taken in the queue.
    +
    +#### System Bolt (Worker) Metrics
    +
    +The System Bolt `__system` provides lots of metrics for different worker wide things.  The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above.
    +
    +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it.
    +
    +##### Receive (NettyServer)
    +`__recv-iconnection` under the `__system` bolt reports stats for the netty server on the worker.  This is what gets messages from other workers.  It is of the form
    +
    +```
    +{
    +    "dequeuedMessages": 0,
    +    "enqueued": {
    +      "/127.0.0.1:49952": 389951
    +    }
    +}
    +```
    +
    +`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts.  That is no longer the case and the value can be ignored.
    +`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker.
    +
    +##### Send (Netty Client)
    +
    +The `__send-iconnection` metric of the `__system` bolt holds information about all of the clients for this worker that are sending metrics.  It is of the form
    --- End diff --
    
    Not sure I understand this. Are there worker clients not shown in this list?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126519537
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    --- End diff --
    
    Nit: For spouts it is the number of tuple trees that were fully acked


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm issue #2200: STORM-2616: Documentation for built in metrics

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the issue:

    https://github.com/apache/storm/pull/2200
  
    +1 Nice documentation!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126547602
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    --- End diff --
    
    If bucket.size.secs is configured to be too small, the update to metrics consumers will happen more frequently and it could make some metrics consumers being overloaded.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126551578
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    --- End diff --
    
    Sure, but wouldn't that just be a performance issue? I'm referring to "and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off", which seems to say that some consumers will report the wrong results if the bucket size is changed(?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126515387
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    --- End diff --
    
    state -> stated. Also is the critical path a concept mentioned elsewhere? I'm worried a new user won't know what that means.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/2200


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126520704
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    --- End diff --
    
    metrics -> metric. Also "a bolt's `execute` method on a bolt" is redundant


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126523893
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    --- End diff --
    
    Wondering if it would be clearer to replace "entry" with "slot" here for consistency. Is NOTE supposed to be all caps? Shouldn't it say "For the `__sendqueue` queue" instead? Missing period after "and not tuples". I had a little trouble understanding the last half of this. Would "In older versions of Storm all of the queue metrics counted slots in the queue. This has been changed in version x.y.z so the queue metrics now always count tuples. Please be careful when trying to compare metrics between different versions of Storm" be accurate? If so I think it's maybe easier to understand.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126516022
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    --- End diff --
    
    nit: "generally reported" -> "generally reported to metrics consumers"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126521238
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    --- End diff --
    
    but -> and


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126532926
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    +
    +`arrival_rate_secs` is an estimation of the number of tuple that are inserted into the queue in one second, although it is actually the dequeue rate.
    +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each entry sits in the queue before it is processed.
    +
    +A disruptor queue has a set number of slots.  If the regular queue fills up an overflow queue takes over.  The number of tuple batches stored in this overflow section are represented by the `overflow` metric.  Storm also does some micro batching of tuples for performance/efficiency reasons so you may see the overflow with a very small number in it even if the queue is not full.
    +
    +`read_pos` and `write_pos` are internal disruptor accounting numbers.  You can think of them almost as the total number of tuple batches written (`write_pos`) or read (`read_pos`) since the queue was created.  They allow for integer overflow so if you use them please take that into account.
    +
    +`capacity` is the total number of slots in the queue. `population` is the number of slots taken in the queue.
    +
    +#### System Bolt (Worker) Metrics
    +
    +The System Bolt `__system` provides lots of metrics for different worker wide things.  The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above.
    +
    +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it.
    +
    +##### Receive (NettyServer)
    +`__recv-iconnection` under the `__system` bolt reports stats for the netty server on the worker.  This is what gets messages from other workers.  It is of the form
    +
    +```
    +{
    +    "dequeuedMessages": 0,
    +    "enqueued": {
    +      "/127.0.0.1:49952": 389951
    +    }
    +}
    +```
    +
    +`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts.  That is no longer the case and the value can be ignored.
    +`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker.
    +
    +##### Send (Netty Client)
    +
    +The `__send-iconnection` metric of the `__system` bolt holds information about all of the clients for this worker that are sending metrics.  It is of the form
    +
    +```
    +{
    +    NodeInfo(node:7decee4b-c314-41f4-b362-fd1358c985b3-127.0.01, port:[6701]): {
    +        "reconnects": 0,
    +        "src": "/127.0.0.1:49951",
    +        "pending": 0,
    +        "dest": "localhost/127.0.0.1:6701",
    +        "sent": 420779,
    +        "lostOnSend": 0
    +    }
    +}
    +```
    +
    +The value is a map where the key is a NodeInfo class for the downstream worker it is sending messages to.  This is the SupervisorId + port.  The value is another map with the fields
    +
    +`src`  What host/port this client has used to connect to the receiving worker.
    --- End diff --
    
    Maybe this should be a list, so it's rendered one value per line. Github is appending all these to the same line. Same issue in the JVM Memory section, the GC section and the uptime section


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126515274
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    --- End diff --
    
    Isn't `topology.builtin.metrics.bucket.size.secs` a configuration parameter, not a metric?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126528804
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    +
    +`arrival_rate_secs` is an estimation of the number of tuple that are inserted into the queue in one second, although it is actually the dequeue rate.
    +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each entry sits in the queue before it is processed.
    +
    +A disruptor queue has a set number of slots.  If the regular queue fills up an overflow queue takes over.  The number of tuple batches stored in this overflow section are represented by the `overflow` metric.  Storm also does some micro batching of tuples for performance/efficiency reasons so you may see the overflow with a very small number in it even if the queue is not full.
    +
    +`read_pos` and `write_pos` are internal disruptor accounting numbers.  You can think of them almost as the total number of tuple batches written (`write_pos`) or read (`read_pos`) since the queue was created.  They allow for integer overflow so if you use them please take that into account.
    +
    +`capacity` is the total number of slots in the queue. `population` is the number of slots taken in the queue.
    +
    +#### System Bolt (Worker) Metrics
    +
    +The System Bolt `__system` provides lots of metrics for different worker wide things.  The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above.
    +
    +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it.
    +
    +##### Receive (NettyServer)
    +`__recv-iconnection` under the `__system` bolt reports stats for the netty server on the worker.  This is what gets messages from other workers.  It is of the form
    --- End diff --
    
    Nit: If all these metrics are all for the `__system` bolt, that part of the description can be omitted since they are under the system bolt header.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request #2200: STORM-2616: Documentation for built in metrics

Posted by srdo <gi...@git.apache.org>.
Github user srdo commented on a diff in the pull request:

    https://github.com/apache/storm/pull/2200#discussion_r126529184
  
    --- Diff: docs/Metrics.md ---
    @@ -125,3 +126,193 @@ The [builtin metrics]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/sto
     
     [BuiltinMetricsUtil.java]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java) sets up data structures for the built-in metrics, and facade methods that the other framework components can use to update them. The metrics themselves are calculated in the calling code -- see for example [`ackSpoutMsg`]({{page.git-blob-base}}/storm-client/src/jvm/org/apache/storm/executor/Executor.java).
     
    +#### Reporting Rate
    +
    +The rate at which built in metrics are reported is configurable through the `topology.builtin.metrics.bucket.size.secs` metric.  If you set this too low it can overload the consumers
    +and some metrics consumers expect metrics to show up at a fixed rate or the numbers could be off, so please use caution when modifying this.
    +
    +
    +#### Tuple Counting Metrics
    +
    +There are several different metrics related to counting what a bolt or spout does to a tuple. These include things like emitting, transferring, acking, and failing of tuples.
    +
    +In general all of these tuple count metrics are randomly sub-sampled unless otherwise state.  This means that the counts you see both on the UI and from the built in metrics are not necessarily exact.  In fact by default we sample only 5% of the events and estimate the total number of events from that.  The sampling percentage is configurable per topology through the `topology.stats.sample.rate` config.  Setting it to 1.0 will make the counts exact, but be aware that the more events we sample the slower your topology will run (as the metrics are counted on the critical path).  This is why we have a 5% sample rate as the default.
    +
    +The tuple counting metrics are generally reported as maps unless explicitly stated otherwise.  They break down each count for finer grained reporting.
    +The keys to these maps fall into two categories `"${stream_name}"` or `"${upstream_component}:${stream_name}"`.  The former is used for all spout metrics and for outgoing bolt metrics (`__emit-count` and `__transfer-count`).  The later is used for bolt metrics that deal with incoming tuples.
    +
    +So for a word count topology the count bolt might show something like the following for an `__ack-count` metrics
    +
    +```
    +{
    +    "split:default": 80080
    +}
    +```
    +
    +But the spout would show something more like for the same metric.
    +
    +```
    +{
    +    "default": 12500
    +}
    +```
    +
    +
    +##### `__ack-count`
    +
    +For bolts it is the number of incoming tuples that had the `ack` method called on them.  For spouts it is the number of tuples that were fully acked.  If acking is disabled this metric is still reported, but it is not really meaningful.
    +
    +##### `__fail-count`
    +
    +For bolts this is the number of incoming tuples that had the `fail` method called on them.  For spouts this is the number of tuples that failed.  It could be because of a tuple timing out or it could be because a bolt called fail on it.  The two are not separated out.
    +
    +##### `__emit-count`
    +
    +This is the total number of times the `emit` method was called to send a tuple.  This is the same for both bolts and spouts.
    +
    +##### `__transfer-count`
    +
    +This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match `__emit_count`.  If nothing is registered to receive a tuple down stream the number will be 0 even if tuples were emitted.  Similarly if there are multiple down stream consumers it may be a multiple of the number emitted.  The grouping also can play a role if it sends the tuple to multiple instances of a single bolt down stream.
    +
    +##### `__execute-count`
    +
    +This count metrics is bolt specific.  It counts the number of times that a bolt's `execute` method on a bolt was called.
    +
    +#### Tuple Latency Metrics
    +
    +Similar to the tuple counting metrics storm also collects average latency metrics for bolts and spouts.  These follow the same structure as the bolt/spout maps and are sub-sampled in the same way as well.  In all cases the latency is measured in milliseconds.
    +
    +##### `__complete-latency`
    +
    +The complete latency is just for spouts.  It is the average amount of time it took for `ack` or `fail` to be called for a tuple after it was emitted.  If acking is disabled this metric is likely to be blank or 0 for all values, but should be ignored.
    +
    +##### `__execute-latency`
    +
    +This is just for bolts.  It is the average amount of time that the bolt spent in the call to the `execute` method.  The longer this gets the fewer tuples a single bolt instance can process.
    +
    +##### `__process-latency`
    +
    +This is also just for bolts.  It is the average amount of time between when `execute` was called to start processing a tuple, to when it was acked or failed by the bolt.  If your bolt is a very simple bolt and the processing is synchronous then `__process-latency` and `__execute-latency` should be very close to one another, with process latency being slightly smaller.  If you are doing a join or have asynchronous processing then it may take a while for a tuple to be acked so the process latency would be higher than the execute latency.
    +
    +##### `__skipped-max-spout-ms`
    +
    +This metric records how much time a spout was idle because more tuples than `topology.max.spout.pending` were still outstanding.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +
    +##### `__skipped-throttle-ms`
    +
    +This metric records how much time a spout was idle because back-pressure indicated that downstream queues in the topology were too full.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +##### `skipped-inactive-ms`
    +
    +This metric records how much time a spout was idle because the topology was deactivated.  This is the total time in milliseconds, not the average amount of time and is not sub-sampled.
    +
    +#### Queue Metrics
    +
    +Each bolt or spout instance in a topology has a receive queue and a send queue.  Each worker also has a queue for sending messages to other workers.  All of these have metrics that are reported.
    +
    +The receive queue metrics are reported under the `__receive` name and send queue metrics are reported under the `__sendqueue` for the given bolt/spout they are a part of.  The metrics for the queue that sends messages to other workers is under the `__transfer` metric name for the system bolt (`__system`).
    +
    +They all have the form.
    +
    +```
    +{
    +    "arrival_rate_secs": 1229.1195171893523,
    +    "overflow": 0,
    +    "read_pos": 103445,
    +    "write_pos": 103448,
    +    "sojourn_time_ms": 2.440771591407277,
    +    "capacity": 1024,
    +    "population": 19
    +}
    +```
    +
    +NOTE that in the `__receive` and `__transfer` queues a single entry may hold 1 or more tuples in it.  For the `__sendqueue` metrics each slot holds a single tuple.  The batching is an optimization that has been in storm since the beginning, so be careful with how you interpret the metrics.  In older versions of storm all of the metrics represent slots in the queue, and not tuples That has been updated so please be careful when trying to compare metrics between different versions of storm.
    +
    +`arrival_rate_secs` is an estimation of the number of tuple that are inserted into the queue in one second, although it is actually the dequeue rate.
    +The `sojourn_time_ms` is calculated from the arrival rate and is an estimate of how many milliseconds each entry sits in the queue before it is processed.
    +
    +A disruptor queue has a set number of slots.  If the regular queue fills up an overflow queue takes over.  The number of tuple batches stored in this overflow section are represented by the `overflow` metric.  Storm also does some micro batching of tuples for performance/efficiency reasons so you may see the overflow with a very small number in it even if the queue is not full.
    +
    +`read_pos` and `write_pos` are internal disruptor accounting numbers.  You can think of them almost as the total number of tuple batches written (`write_pos`) or read (`read_pos`) since the queue was created.  They allow for integer overflow so if you use them please take that into account.
    +
    +`capacity` is the total number of slots in the queue. `population` is the number of slots taken in the queue.
    +
    +#### System Bolt (Worker) Metrics
    +
    +The System Bolt `__system` provides lots of metrics for different worker wide things.  The one metric not described here is the `__transfer` queue metric, because it fits with the other disruptor metrics described above.
    +
    +Be aware that the `__system` bolt is an actual bolt so regular bolt metrics described above also will be reported for it.
    +
    +##### Receive (NettyServer)
    +`__recv-iconnection` under the `__system` bolt reports stats for the netty server on the worker.  This is what gets messages from other workers.  It is of the form
    +
    +```
    +{
    +    "dequeuedMessages": 0,
    +    "enqueued": {
    +      "/127.0.0.1:49952": 389951
    +    }
    +}
    +```
    +
    +`dequeuedMessages` is a throwback to older code where there was an internal queue between the server and the bolts/spouts.  That is no longer the case and the value can be ignored.
    +`enqueued` is a map between the address of the remote worker and the number of tuples that were sent from it to this worker.
    +
    +##### Send (Netty Client)
    +
    +The `__send-iconnection` metric of the `__system` bolt holds information about all of the clients for this worker that are sending metrics.  It is of the form
    +
    +```
    +{
    +    NodeInfo(node:7decee4b-c314-41f4-b362-fd1358c985b3-127.0.01, port:[6701]): {
    +        "reconnects": 0,
    +        "src": "/127.0.0.1:49951",
    +        "pending": 0,
    +        "dest": "localhost/127.0.0.1:6701",
    +        "sent": 420779,
    +        "lostOnSend": 0
    +    }
    +}
    +```
    +
    +The value is a map where the key is a NodeInfo class for the downstream worker it is sending messages to.  This is the SupervisorId + port.  The value is another map with the fields
    +
    +`src`  What host/port this client has used to connect to the receiving worker.
    +`dest` What host/port this client has connected to.
    +`reconnects` the number of reconnections that have happened.
    +`pending` the number of messages that have not been sent.  (This corresponds to messages, not tuples)
    +`sent` the number of messages that have been send.  (This is messages not tuples)
    --- End diff --
    
    send -> sent


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---