blob: 5c35dd77a71ee5c577298c3ee2d473bcdfc50f5b [file] [log] [blame]
<!DOCTYPE html>
<html>
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge">
<meta name="viewport" content="width=device-width, initial-scale=1">
<link rel="shortcut icon" href="/favicon.ico" type="image/x-icon">
<link rel="icon" href="/favicon.ico" type="image/x-icon">
<title>Storm Metrics</title>
<!-- Bootstrap core CSS -->
<link href="/assets/css/bootstrap.min.css" rel="stylesheet">
<!-- Bootstrap theme -->
<link href="/assets/css/bootstrap-theme.min.css" rel="stylesheet">
<!-- Custom styles for this template -->
<link rel="stylesheet" href="http://fortawesome.github.io/Font-Awesome/assets/font-awesome/css/font-awesome.css">
<link href="/css/style.css" rel="stylesheet">
<link href="/assets/css/owl.theme.css" rel="stylesheet">
<link href="/assets/css/owl.carousel.css" rel="stylesheet">
<script type="text/javascript" src="/assets/js/jquery.min.js"></script>
<script type="text/javascript" src="/assets/js/bootstrap.min.js"></script>
<script type="text/javascript" src="/assets/js/owl.carousel.min.js"></script>
<script type="text/javascript" src="/assets/js/storm.js"></script>
<!-- Just for debugging purposes. Don't actually copy these 2 lines! -->
<!--[if lt IE 9]><script src="../../assets/js/ie8-responsive-file-warning.js"></script><![endif]-->
<!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
<!--[if lt IE 9]>
<script src="https://oss.maxcdn.com/html5shiv/3.7.2/html5shiv.min.js"></script>
<script src="https://oss.maxcdn.com/respond/1.4.2/respond.min.js"></script>
<![endif]-->
</head>
<body>
<header>
<div class="container-fluid">
<div class="row">
<div class="col-md-5">
<a href="/index.html"><img src="/images/logo.png" class="logo" /></a>
</div>
<div class="col-md-5">
<h1>Version: 2.3.0</h1>
</div>
<div class="col-md-2">
<a href="/downloads.html" class="btn-std btn-block btn-download">Download</a>
</div>
</div>
</div>
</header>
<!--Header End-->
<!--Navigation Begin-->
<div class="navbar" role="banner">
<div class="container-fluid">
<div class="navbar-header">
<button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bs-navbar-collapse">
<span class="icon-bar"></span>
<span class="icon-bar"></span>
<span class="icon-bar"></span>
</button>
</div>
<nav class="collapse navbar-collapse bs-navbar-collapse" role="navigation">
<ul class="nav navbar-nav">
<li><a href="/index.html" id="home">Home</a></li>
<li><a href="/getting-help.html" id="getting-help">Getting Help</a></li>
<li><a href="/about/integrates.html" id="project-info">Project Information</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="documentation">Documentation <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/releases/2.3.0/index.html">2.3.0</a></li>
<li><a href="/releases/2.2.0/index.html">2.2.0</a></li>
<li><a href="/releases/2.1.0/index.html">2.1.0</a></li>
<li><a href="/releases/2.0.0/index.html">2.0.0</a></li>
<li><a href="/releases/1.2.3/index.html">1.2.3</a></li>
</ul>
</li>
<li><a href="/talksAndVideos.html">Talks and Slideshows</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="contribute">Community <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/contribute/Contributing-to-Storm.html">Contributing</a></li>
<li><a href="/contribute/People.html">People</a></li>
<li><a href="/contribute/BYLAWS.html">ByLaws</a></li>
</ul>
</li>
<li><a href="/2021/09/27/storm230-released.html" id="news">News</a></li>
</ul>
</nav>
</div>
</div>
<div class="container-fluid">
<h1 class="page-title">Storm Metrics</h1>
<div class="row">
<div class="col-md-12">
<!-- Documentation -->
<p class="post-meta"></p>
<div class="documentation-content"><p>Storm exposes a metrics interface to report summary statistics across the full topology.
The numbers you see on the UI come from some of these built in metrics, but are reported through the worker heartbeats instead of through the IMetricsConsumer described below.</p>
<p>If you are looking for cluster wide monitoring please see <a href="ClusterMetrics.html">Cluster Metrics</a>.</p>
<h3 id="metric-types">Metric Types</h3>
<p>Metrics have to implement <a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/IMetric.java"><code>IMetric</code></a> which contains just one method, <code>getValueAndReset</code> -- do any remaining work to find the summary value, and reset back to an initial state. For example, the MeanReducer divides the running total by its running count to find the mean, then initializes both values back to zero.</p>
<p>Storm gives you these metric types:</p>
<ul>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/AssignableMetric.java">AssignableMetric</a> -- set the metric to the explicit value you supply. Useful if it&#39;s an external value or in the case that you are already calculating the summary statistic yourself.</li>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/CombinedMetric.java">CombinedMetric</a> -- generic interface for metrics that can be updated associatively. </li>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/CountMetric.java">CountMetric</a> -- a running total of the supplied values. Call <code>incr()</code> to increment by one, <code>incrBy(n)</code> to add/subtract the given number.
<ul>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/MultiCountMetric.java">MultiCountMetric</a> -- a hashmap of count metrics.</li>
</ul></li>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/ReducedMetric.java">ReducedMetric</a>
<ul>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/MeanReducer.java">MeanReducer</a> -- track a running average of values given to its <code>reduce()</code> method. (It accepts <code>Double</code>, <code>Integer</code> or <code>Long</code> values, and maintains the internal average as a <code>Double</code>.) Despite his reputation, the MeanReducer is actually a pretty nice guy in person.</li>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/MultiReducedMetric.java">MultiReducedMetric</a> -- a hashmap of reduced metrics.</li>
</ul></li>
</ul>
<p>Be aware that even though <code>getValueAndReset</code> can return an object returning any object makes it very difficult for an <code>IMetricsConsumer</code> to know how to translate it into something usable. Also note that because it is sent to the <code>IMetricsConsumer</code> as a part of a tuple the values returned need to be able to be <a href="Serialization.html">serialized</a> by your topology.</p>
<h3 id="metrics-consumer">Metrics Consumer</h3>
<p>You can listen and handle the topology metrics via registering Metrics Consumer to your topology. </p>
<p>To register metrics consumer to your topology, add to your topology&#39;s configuration like:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">conf</span><span class="o">.</span><span class="na">registerMetricsConsumer</span><span class="o">(</span><span class="n">org</span><span class="o">.</span><span class="na">apache</span><span class="o">.</span><span class="na">storm</span><span class="o">.</span><span class="na">metric</span><span class="o">.</span><span class="na">LoggingMetricsConsumer</span><span class="o">.</span><span class="na">class</span><span class="o">,</span> <span class="mi">1</span><span class="o">);</span>
</code></pre></div>
<p>You can refer <a href="javadocs/org/apache/storm/Config.html#registerMetricsConsumer-java.lang.Class-">Config#registerMetricsConsumer</a> and overloaded methods from javadoc.</p>
<p>Otherwise edit the storm.yaml config file:</p>
<div class="highlight"><pre><code class="language-yaml" data-lang="yaml"><span class="s">topology.metrics.consumer.register</span><span class="pi">:</span>
<span class="pi">-</span> <span class="na">class</span><span class="pi">:</span> <span class="s2">"</span><span class="s">org.apache.storm.metric.LoggingMetricsConsumer"</span>
<span class="s">parallelism.hint</span><span class="pi">:</span> <span class="s">1</span>
<span class="pi">-</span> <span class="na">class</span><span class="pi">:</span> <span class="s2">"</span><span class="s">org.apache.storm.metric.HttpForwardingMetricsConsumer"</span>
<span class="s">parallelism.hint</span><span class="pi">:</span> <span class="s">1</span>
<span class="na">argument</span><span class="pi">:</span> <span class="s2">"</span><span class="s">http://example.com:8080/metrics/my-topology/"</span>
</code></pre></div>
<p>Storm adds a MetricsConsumerBolt to your topolology for each class in the <code>topology.metrics.consumer.register</code> list. Each MetricsConsumerBolt subscribes to receive metrics from all tasks in the topology. The parallelism for each Bolt is set to <code>parallelism.hint</code> and <code>component id</code> for that Bolt is set to <code>__metrics_&lt;metrics consumer class name&gt;</code>. If you register the same class name more than once, postfix <code>#&lt;sequence number&gt;</code> is appended to component id.</p>
<p>Storm provides some built-in metrics consumers for you to try out to see which metrics are provided in your topology.</p>
<ul>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/LoggingMetricsConsumer.java"><code>LoggingMetricsConsumer</code></a> -- listens for all metrics and dumps them to log file with TSV (Tab Separated Values).</li>
<li><a href="http://github.com/apache/storm/blob/v2.3.0/examples/storm-loadgen/src/main/java/org/apache/storm/loadgen/HttpForwardingMetricsConsumer.java"><code>HttpForwardingMetricsConsumer</code></a> -- listens for all metrics and POSTs them serialized to a configured URL via HTTP. Storm also provides <a href="http://github.com/apache/storm/blob/v2.3.0/storm-core/src/jvm/org/apache/storm/metric/HttpForwardingMetricsServer.java"><code>HttpForwardingMetricsServer</code></a> as abstract class so you can extend this class and run as a HTTP server, and handle metrics sent by HttpForwardingMetricsConsumer.</li>
</ul>
<p>Also, Storm exposes the interface <a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/metric/api/IMetricsConsumer.java"><code>IMetricsConsumer</code></a> for implementing Metrics Consumer so you can create custom metrics consumers and attach to their topologies, or use other great implementation of Metrics Consumers provided by Storm community. Some of examples are <a href="https://github.com/verisign/storm-graphite">versign/storm-graphite</a>, and <a href="https://github.com/endgameinc/storm-metrics-statsd">storm-metrics-statsd</a>.</p>
<p>When you implement your own metrics consumer, <code>argument</code> is passed to Object when <a href="javadocs/org/apache/storm/metric/api/IMetricsConsumer.html#prepare-java.util.Map-java.lang.Object-org.apache.storm.task.TopologyContext-org.apache.storm.task.IErrorReporter-">IMetricsConsumer#prepare</a> is called, so you need to infer the Java type of configured value on yaml, and do explicit type casting.</p>
<p>Please keep in mind that MetricsConsumerBolt is just a kind of Bolt, so whole throughput of the topology will go down when registered metrics consumers cannot keep up handling incoming metrics, so you may want to take care of those Bolts like normal Bolts. One of idea to avoid this is making your implementation of Metrics Consumer as <code>non-blocking</code> fashion.</p>
<h3 id="build-your-own-metric-task-level">Build your own metric (task level)</h3>
<p>You can measure your own metric by registering <code>IMetric</code> to Metric Registry. </p>
<p>Suppose we would like to measure execution count of Bolt#execute. Let&#39;s start with defining metric instance. CountMetric seems to fit our use case.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">private</span> <span class="kd">transient</span> <span class="n">CountMetric</span> <span class="n">countMetric</span><span class="o">;</span>
</code></pre></div>
<p>Notice we define it as transient. IMertic is not Serializable so we defined as transient to avoid any serialization issues.</p>
<p>Next, let&#39;s initialize and register the metric instance.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">prepare</span><span class="o">(</span><span class="n">Map</span> <span class="n">conf</span><span class="o">,</span> <span class="n">TopologyContext</span> <span class="n">context</span><span class="o">,</span> <span class="n">OutputCollector</span> <span class="n">collector</span><span class="o">)</span> <span class="o">{</span>
<span class="c1">// other initialization here.</span>
<span class="n">countMetric</span> <span class="o">=</span> <span class="k">new</span> <span class="n">CountMetric</span><span class="o">();</span>
<span class="n">context</span><span class="o">.</span><span class="na">registerMetric</span><span class="o">(</span><span class="s">"execute_count"</span><span class="o">,</span> <span class="n">countMetric</span><span class="o">,</span> <span class="mi">60</span><span class="o">);</span>
<span class="o">}</span>
</code></pre></div>
<p>The meaning of first and second parameters are straightforward, metric name and instance of IMetric. Third parameter of <a href="javadocs/org/apache/storm/task/TopologyContext.html#registerMetric-java.lang.String-T-int-">TopologyContext#registerMetric</a> is the period (seconds) to publish and reset the metric.</p>
<p>Last, let&#39;s increment the value when Bolt.execute() is executed.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kt">void</span> <span class="nf">execute</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">input</span><span class="o">)</span> <span class="o">{</span>
<span class="n">countMetric</span><span class="o">.</span><span class="na">incr</span><span class="o">();</span>
<span class="c1">// handle tuple here. </span>
<span class="o">}</span>
</code></pre></div>
<p>Note that sample rate for topology metrics is not applied to custom metrics since we&#39;re calling incr() ourselves.</p>
<p>Done! <code>countMetric.getValueAndReset()</code> is called every 60 seconds as we registered as period, and pair of (&quot;execute_count&quot;, value) will be pushed to MetricsConsumer.</p>
<h3 id="build-your-own-metrics-worker-level">Build your own metrics (worker level)</h3>
<p>You can register your own worker level metrics by adding them to <code>Config.WORKER_METRICS</code> for all workers in cluster, or <code>Config.TOPOLOGY_WORKER_METRICS</code> for all workers in specific topology.</p>
<p>For example, we can add <code>worker.metrics</code> to storm.yaml in cluster,</p>
<div class="highlight"><pre><code class="language-yaml" data-lang="yaml"><span class="s">worker.metrics</span><span class="pi">:</span>
<span class="na">metricA</span><span class="pi">:</span> <span class="s2">"</span><span class="s">aaa.bbb.ccc.ddd.MetricA"</span>
<span class="na">metricB</span><span class="pi">:</span> <span class="s2">"</span><span class="s">aaa.bbb.ccc.ddd.MetricB"</span>
<span class="s">...</span>
</code></pre></div>
<p>or put <code>Map&lt;String, String&gt;</code> (metric name, metric class name) with key <code>Config.TOPOLOGY_WORKER_METRICS</code> to config map.</p>
<p>There are some restrictions for worker level metric instances: </p>
<p>A) Metrics for worker level should be kind of gauge since it is initialized and registered from SystemBolt and not exposed to user tasks.</p>
<p>B) Metrics will be initialized with default constructor, and no injection for configuration or object will be performed.</p>
<p>C) Bucket size (seconds) for metrics is fixed to <code>Config.TOPOLOGY_BUILTIN_METRICS_BUCKET_SIZE_SECS</code>.</p>
<h3 id="builtin-metrics">Builtin Metrics</h3>
<p>The <a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java">builtin metrics</a> instrument Storm itself.</p>
<p><a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/daemon/metrics/BuiltinMetricsUtil.java">BuiltinMetricsUtil.java</a> 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 <a href="http://github.com/apache/storm/blob/v2.3.0/storm-client/src/jvm/org/apache/storm/executor/Executor.java"><code>ackSpoutMsg</code></a>.</p>
<h4 id="reporting-rate">Reporting Rate</h4>
<p>The rate at which built in metrics are reported is configurable through the <code>topology.builtin.metrics.bucket.size.secs</code> config. If you set this too low it can overload the consumers,
so please use caution when modifying it.</p>
<h4 id="tuple-counting-metrics">Tuple Counting Metrics</h4>
<p>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.</p>
<p>In general all of these tuple count metrics are randomly sub-sampled unless otherwise stated. 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 <code>topology.stats.sample.rate</code> 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 in the same code path as tuples are processed). This is why we have a 5% sample rate as the default.</p>
<p>The tuple counting metric names contain <code>&quot;${stream_name}&quot;</code> or <code>&quot;${upstream_component}:${stream_name}&quot;</code>. The former is used for all spout metrics and for outgoing bolt metrics (<code>__emit-count</code> and <code>__transfer-count</code>). The latter is used for bolt metrics that deal with incoming tuples.</p>
<p>So for a word count topology the count bolt might show something like the following for the <code>__ack-count</code> metric</p>
<div class="highlight"><pre><code class="language-" data-lang=""> "__ack-count-split:default": 80080
</code></pre></div>
<p>But the spout instead would show something like the following for the <code>__ack-count</code> metric.</p>
<div class="highlight"><pre><code class="language-" data-lang=""> "__ack-count-default": 12500
</code></pre></div>
<h5 id="__ack-count"><code>__ack-count</code></h5>
<p>For bolts it is the number of incoming tuples that had the <code>ack</code> method called on them. For spouts it is the number of tuples trees that were fully acked. See Guaranteeing Message Processing<a href="Guaranteeing-message-processing.html"></a> for more information about what a tuple tree is. If acking is disabled this metric is still reported, but it is not really meaningful.</p>
<h5 id="__fail-count"><code>__fail-count</code></h5>
<p>For bolts this is the number of incoming tuples that had the <code>fail</code> method called on them. For spouts this is the number of tuple trees that failed. Tuple trees may fail from timing out or because a bolt called fail on it. The two are not separated out by this metric.</p>
<h5 id="__emit-count"><code>__emit-count</code></h5>
<p>This is the total number of times the <code>emit</code> method was called to send a tuple. This is the same for both bolts and spouts.</p>
<h5 id="__transfer-count"><code>__transfer-count</code></h5>
<p>This is the total number of tuples transferred to a downstream bolt/spout for processing. This number will not always match <code>__emit_count</code>. 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.</p>
<h5 id="__execute-count"><code>__execute-count</code></h5>
<p>This count metric is bolt specific. It counts the number of times that a bolt&#39;s <code>execute</code> method was called.</p>
<h4 id="tuple-latency-metrics">Tuple Latency Metrics</h4>
<p>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.</p>
<h5 id="__complete-latency"><code>__complete-latency</code></h5>
<p>The complete latency is just for spouts. It is the average amount of time it took for <code>ack</code> or <code>fail</code> 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, and should be ignored.</p>
<h5 id="__execute-latency"><code>__execute-latency</code></h5>
<p>This is just for bolts. It is the average amount of time that the bolt spent in the call to the <code>execute</code> method. The higher this gets, the lower the throughput of tuples per bolt instance.</p>
<h5 id="__process-latency"><code>__process-latency</code></h5>
<p>This is also just for bolts. It is the average amount of time between when <code>execute</code> 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 <code>__process-latency</code> and <code>__execute-latency</code> 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.</p>
<h5 id="__skipped-max-spout-ms"><code>__skipped-max-spout-ms</code></h5>
<p>This metric records how much time a spout was idle because more tuples than <code>topology.max.spout.pending</code> were still outstanding. This is the total time in milliseconds, not the average amount of time and is not sub-sampled.</p>
<h5 id="__skipped-backpressure-ms"><code>__skipped-backpressure-ms</code></h5>
<p>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. This is similar to skipped-throttle-ms in Storm 1.x.</p>
<h5 id="__backpressure-last-overflow-count"><code>__backpressure-last-overflow-count</code></h5>
<p>This metric indicates the overflow count last time BP status was sent, with a minimum value of 1 if a task has backpressure on.</p>
<h5 id="skipped-inactive-ms"><code>skipped-inactive-ms</code></h5>
<p>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.</p>
<h4 id="error-reporting-metrics">Error Reporting Metrics</h4>
<p>Storm also collects error reporting metrics for bolts and spouts.</p>
<h5 id="__reported-error-count"><code>__reported-error-count</code></h5>
<p>This metric records how many errors were reported by a spout/bolt. It is the total number of times the <code>reportError</code> method was called.</p>
<h4 id="queue-metrics">Queue Metrics</h4>
<p>Each bolt or spout instance in a topology has a receive queue. Each worker also has a worker transfer queue for sending messages to other workers. All of these have metrics that are reported.</p>
<p>The receive queue metrics are reported under the <code>receive_queue</code> name. The metrics for the queue that sends messages to other workers is under the <code>worker-transfer-queue</code> metric name for the system bolt (<code>__system</code>).</p>
<p>These queues report the following metrics:</p>
<div class="highlight"><pre><code class="language-" data-lang="">{
"arrival_rate_secs": 1229.1195171893523,
"overflow": 0,
"sojourn_time_ms": 2.440771591407277,
"capacity": 1024,
"population": 19,
"pct_full": "0.018".
"insert_failures": "0",
"dropped_messages": "0"
}
</code></pre></div>
<p><code>arrival_rate_secs</code> is an estimation of the number of tuples that are inserted into the queue in one second, although it is actually the dequeue rate.
The <code>sojourn_time_ms</code> is calculated from the arrival rate and is an estimate of how many milliseconds each tuple sits in the queue before it is processed.</p>
<p>The queue has a set maximum number of entries. If the regular queue fills up an overflow queue takes over. The number of tuples stored in this overflow section are represented by the <code>overflow</code> metric. Note that an overflow queue is only used for executors to receive tuples from remote workers. It doesn&#39;t apply to intra-worker tuple transfer.</p>
<p><code>capacity</code> is the maximum number of entries in the queue. <code>population</code> is the number of entries currently filled in the queue. &#39;pct_full&#39; tracks the percentage of capacity in use.</p>
<p>&#39;insert_failures&#39; tracks the number of failures inserting into the queue. &#39;dropped_messages&#39; tracks messages dropped due to the overflow queue being full.</p>
<h4 id="system-bolt-worker-metrics">System Bolt (Worker) Metrics</h4>
<p>The System Bolt <code>__system</code> provides lots of metrics for different worker wide things. The one metric not described here is the <code>__transfer</code> queue metric, because it fits with the other disruptor metrics described above.</p>
<p>Be aware that the <code>__system</code> bolt is an actual bolt so regular bolt metrics described above also will be reported for it.</p>
<h5 id="receive-nettyserver">Receive (NettyServer)</h5>
<p><code>__recv-iconnection</code> reports stats for the netty server on the worker. This is what gets messages from other workers. It is of the form</p>
<div class="highlight"><pre><code class="language-" data-lang="">{
"dequeuedMessages": 0,
"enqueued": {
"/127.0.0.1:49952": 389951
}
}
</code></pre></div>
<p><code>dequeuedMessages</code> 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.
<code>enqueued</code> is a map between the address of the remote worker and the number of tuples that were sent from it to this worker.</p>
<h5 id="send-netty-client">Send (Netty Client)</h5>
<p>The <code>__send-iconnection</code> metrics report information about all of the clients for this worker. They are named __send-iconnection-METRIC_TYPE-HOST:PORT for a given Client that is
connected to a worker with the given host/port.</p>
<p>The metric types reported for each client are:</p>
<ul>
<li><code>reconnects</code> the number of reconnections that have happened.</li>
<li><code>pending</code> the number of messages that have not been sent. (This corresponds to messages, not tuples)</li>
<li><code>sent</code> the number of messages that have been sent. (This is messages not tuples)</li>
<li><code>lostOnSend</code>. This is the number of messages that were lost because of connection issues. (This is messages not tuples). </li>
</ul>
<h5 id="jvm-memory">JVM Memory</h5>
<p>JVM memory usage is reported through <code>memory.non-heap</code> for off heap memory, <code>memory.heap</code> for on heap memory and <code>memory.total</code> for combined values. These values come from the <a href="https://docs.oracle.com/javase/8/docs/api/index.html?java/lang/management/MemoryUsage.html">MemoryUsage</a> mxbean. Each of the metrics are reported as a map with the following keys, and values returned by the corresponding java code.</p>
<table><thead>
<tr>
<th>Key</th>
<th>Corresponding Code</th>
</tr>
</thead><tbody>
<tr>
<td><code>max</code></td>
<td><code>memUsage.getMax()</code></td>
</tr>
<tr>
<td><code>committed</code></td>
<td><code>memUsage.getCommitted()</code></td>
</tr>
<tr>
<td><code>init</code></td>
<td><code>memUsage.getInit()</code></td>
</tr>
<tr>
<td><code>used</code></td>
<td><code>memUsage.getUsed()</code></td>
</tr>
<tr>
<td><code>usage</code></td>
<td><code>Ratio.of(memUsage.getUsed(), memUsage.getMax())</code></td>
</tr>
</tbody></table>
<h5 id="jvm-garbage-collection">JVM Garbage Collection</h5>
<p>The exact GC metric name depends on the garbage collector that your worker uses. The data is all collected from <code>ManagementFactory.getGarbageCollectorMXBeans()</code> and the name of the metrics is <code>&quot;GC&quot;</code> followed by the name of the returned bean with white space removed. The reported metrics are just</p>
<ul>
<li><code>count</code> the number of gc events that happened and</li>
<li><code>time</code> the total number of milliseconds that were spent doing gc.<br></li>
</ul>
<p>Please refer to the <a href="https://docs.oracle.com/javase/8/docs/api/java/lang/management/ManagementFactory.html#getGarbageCollectorMXBeans--">JVM documentation</a> for more details.</p>
<h5 id="jvm-misc">JVM Misc</h5>
<ul>
<li>There are metrics prefixed with <code>threads</code> providing the number of threads, daemon threads, blocked and deadlocked threads.</li>
</ul>
<h5 id="uptime">Uptime</h5>
<ul>
<li><code>uptimeSecs</code> reports the number of seconds the worker has been up for</li>
<li><code>newWorkerEvent</code> is 1 when a worker is first started and 0 all other times. This can be used to tell when a worker has crashed and is restarted.</li>
<li><code>startTimeSecs</code> is when the worker started in seconds since the epoch</li>
</ul>
<h5 id="doheartbeat-calls">doHeartbeat-calls</h5>
<ul>
<li><code>doHeartbeat-calls</code> is a meter that indicates the rate the worker is performing heartbeats.</li>
</ul>
</div>
</div>
</div>
</div>
<footer>
<div class="container-fluid">
<div class="row">
<div class="col-md-3">
<div class="footer-widget">
<h5>Meetups</h5>
<ul class="latest-news">
<li><a href="http://www.meetup.com/Apache-Storm-Apache-Kafka/">Apache Storm & Apache Kafka</a> <span class="small">(Sunnyvale, CA)</span></li>
<li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Apache Storm & Kafka Users</a> <span class="small">(Seattle, WA)</span></li>
<li><a href="http://www.meetup.com/New-York-City-Storm-User-Group/">NYC Storm User Group</a> <span class="small">(New York, NY)</span></li>
<li><a href="http://www.meetup.com/Bay-Area-Stream-Processing">Bay Area Stream Processing</a> <span class="small">(Emeryville, CA)</span></li>
<li><a href="http://www.meetup.com/Boston-Storm-Users/">Boston Realtime Data</a> <span class="small">(Boston, MA)</span></li>
<li><a href="http://www.meetup.com/storm-london">London Storm User Group</a> <span class="small">(London, UK)</span></li>
<!-- <li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Seatle, WA</a> <span class="small">(27 Jun 2015)</span></li> -->
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>About Apache Storm</h5>
<p>Apache Storm integrates with any queueing system and any database system. Apache Storm's spout abstraction makes it easy to integrate a new queuing system. Likewise, integrating Apache Storm with database systems is easy.</p>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>First Look</h5>
<ul class="footer-list">
<li><a href="/releases/current/Rationale.html">Rationale</a></li>
<li><a href="/releases/current/Tutorial.html">Tutorial</a></li>
<li><a href="/releases/current/Setting-up-development-environment.html">Setting up development environment</a></li>
<li><a href="/releases/current/Creating-a-new-Storm-project.html">Creating a new Apache Storm project</a></li>
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>Documentation</h5>
<ul class="footer-list">
<li><a href="/releases/current/index.html">Index</a></li>
<li><a href="/releases/current/javadocs/index.html">Javadoc</a></li>
<li><a href="/releases/current/FAQ.html">FAQ</a></li>
</ul>
</div>
</div>
</div>
<hr/>
<div class="row">
<div class="col-md-12">
<p align="center">Copyright © 2019 <a href="http://www.apache.org">Apache Software Foundation</a>. All Rights Reserved.
<br>Apache Storm, Apache, the Apache feather logo, and the Apache Storm project logos are trademarks of The Apache Software Foundation.
<br>All other marks mentioned may be trademarks or registered trademarks of their respective owners.</p>
</div>
</div>
</div>
</footer>
<!--Footer End-->
<!-- Scroll to top -->
<span class="totop"><a href="#"><i class="fa fa-angle-up"></i></a></span>
</body>
</html>