blob: 351337844c6d93d7fb10e2f2f776d7c7a029038c [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 UI REST API</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: 1.0.6</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.0.0-SNAPSHOT/index.html">2.0.0-SNAPSHOT</a></li>
<li><a href="/releases/1.2.1/index.html">1.2.1</a></li>
<li><a href="/releases/1.1.2/index.html">1.1.2</a></li>
<li><a href="/releases/1.0.6/index.html">1.0.6</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="/2018/02/19/storm121-released.html" id="news">News</a></li>
</ul>
</nav>
</div>
</div>
<div class="container-fluid">
<h1 class="page-title">Storm UI REST API</h1>
<div class="row">
<div class="col-md-12">
<!-- Documentation -->
<p class="post-meta"></p>
<div class="documentation-content"><p>The Storm UI daemon provides a REST API that allows you to interact with a Storm cluster, which includes retrieving
metrics data and configuration information as well as management operations such as starting or stopping topologies.</p>
<h1 id="data-format">Data format</h1>
<p>The REST API returns JSON responses and supports JSONP.
Clients can pass a callback query parameter to wrap JSON in the callback function.</p>
<h1 id="using-the-ui-rest-api">Using the UI REST API</h1>
<p><em>Note: It is recommended to ignore undocumented elements in the JSON response because future versions of Storm may not</em>
<em>support those elements anymore.</em></p>
<h2 id="rest-api-base-url">REST API Base URL</h2>
<p>The REST API is part of the UI daemon of Storm (started by <code>storm ui</code>) and thus runs on the same host and port as the
Storm UI (the UI daemon is often run on the same host as the Nimbus daemon). The port is configured by <code>ui.port</code>,
which is set to <code>8080</code> by default (see <a href="conf/defaults.yaml">defaults.yaml</a>).</p>
<p>The API base URL would thus be:</p>
<div class="highlight"><pre><code class="language-" data-lang="">http://&lt;ui-host&gt;:&lt;ui-port&gt;/api/v1/...
</code></pre></div>
<p>You can use a tool such as <code>curl</code> to talk to the REST API:</p>
<div class="highlight"><pre><code class="language-" data-lang=""># Request the cluster configuration.
# Note: We assume ui.port is configured to the default value of 8080.
$ curl http://&lt;ui-host&gt;:8080/api/v1/cluster/configuration
</code></pre></div>
<h2 id="impersonating-a-user-in-secure-environment">Impersonating a user in secure environment</h2>
<p>In a secure environment an authenticated user can impersonate another user. To impersonate a user the caller must pass
<code>doAsUser</code> param or header with value set to the user that the request needs to be performed as. Please see SECURITY.MD
to learn more about how to setup impersonation ACLs and authorization. The rest API uses the same configs and acls that
are used by nimbus.</p>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight"> 1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1425844354\?doAsUser=testUSer1
2. curl 'http://localhost:8080/api/v1/topology/wordcount-1-1425844354/activate' -X POST -H 'doAsUser:testUSer1'
</code></pre></div>
<h2 id="get-operations">GET Operations</h2>
<h3 id="api-v1-cluster-configuration-get">/api/v1/cluster/configuration (GET)</h3>
<p>Returns the cluster configuration.</p>
<p>Sample response (does not include all the data fields):</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"dev.zookeeper.path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/tmp/dev-storm-zookeeper"</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.tick.tuple.freq.secs"</span><span class="p">:</span><span class="w"> </span><span class="kc">null</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.builtin.metrics.bucket.size.secs"</span><span class="p">:</span><span class="w"> </span><span class="mi">60</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.fall.back.on.java.serialization"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.max.error.report.per.interval"</span><span class="p">:</span><span class="w"> </span><span class="mi">5</span><span class="p">,</span><span class="w">
</span><span class="s2">"zmq.linger.millis"</span><span class="p">:</span><span class="w"> </span><span class="mi">5000</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.skip.missing.kryo.registrations"</span><span class="p">:</span><span class="w"> </span><span class="kc">false</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.messaging.netty.client_worker_threads"</span><span class="p">:</span><span class="w"> </span><span class="mi">1</span><span class="p">,</span><span class="w">
</span><span class="s2">"ui.childopts"</span><span class="p">:</span><span class="w"> </span><span class="s2">"-Xmx768m"</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.zookeeper.session.timeout"</span><span class="p">:</span><span class="w"> </span><span class="mi">20000</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbus.reassign"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.trident.batch.emit.interval.millis"</span><span class="p">:</span><span class="w"> </span><span class="mi">500</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.messaging.netty.flush.check.interval.ms"</span><span class="p">:</span><span class="w"> </span><span class="mi">10</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbus.monitor.freq.secs"</span><span class="p">:</span><span class="w"> </span><span class="mi">10</span><span class="p">,</span><span class="w">
</span><span class="s2">"logviewer.childopts"</span><span class="p">:</span><span class="w"> </span><span class="s2">"-Xmx128m"</span><span class="p">,</span><span class="w">
</span><span class="s2">"java.library.path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/usr/local/lib:/opt/local/lib:/usr/lib"</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.executor.send.buffer.size"</span><span class="p">:</span><span class="w"> </span><span class="mi">1024</span><span class="p">,</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-cluster-summary-get">/api/v1/cluster/summary (GET)</h3>
<p>Returns cluster summary information such as nimbus uptime or number of supervisors.</p>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>stormVersion</td>
<td>String</td>
<td>Storm version</td>
</tr>
<tr>
<td>supervisors</td>
<td>Integer</td>
<td>Number of supervisors running</td>
</tr>
<tr>
<td>topologies</td>
<td>Integer</td>
<td>Number of topologies running</td>
</tr>
<tr>
<td>slotsTotal</td>
<td>Integer</td>
<td>Total number of available worker slots</td>
</tr>
<tr>
<td>slotsUsed</td>
<td>Integer</td>
<td>Number of worker slots used</td>
</tr>
<tr>
<td>slotsFree</td>
<td>Integer</td>
<td>Number of worker slots available</td>
</tr>
<tr>
<td>executorsTotal</td>
<td>Integer</td>
<td>Total number of executors</td>
</tr>
<tr>
<td>tasksTotal</td>
<td>Integer</td>
<td>Total tasks</td>
</tr>
</tbody></table>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"stormVersion"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.9.2-incubating-SNAPSHOT"</span><span class="p">,</span><span class="w">
</span><span class="s2">"supervisors"</span><span class="p">:</span><span class="w"> </span><span class="mi">1</span><span class="p">,</span><span class="w">
</span><span class="s2">"slotsTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">4</span><span class="p">,</span><span class="w">
</span><span class="s2">"slotsUsed"</span><span class="p">:</span><span class="w"> </span><span class="mi">3</span><span class="p">,</span><span class="w">
</span><span class="s2">"slotsFree"</span><span class="p">:</span><span class="w"> </span><span class="mi">1</span><span class="p">,</span><span class="w">
</span><span class="s2">"executorsTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">28</span><span class="p">,</span><span class="w">
</span><span class="s2">"tasksTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">28</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-supervisor-summary-get">/api/v1/supervisor/summary (GET)</h3>
<p>Returns summary information for all supervisors.</p>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Supervisor&#39;s id</td>
</tr>
<tr>
<td>host</td>
<td>String</td>
<td>Supervisor&#39;s host name</td>
</tr>
<tr>
<td>uptime</td>
<td>String</td>
<td>Shows how long the supervisor is running</td>
</tr>
<tr>
<td>uptimeSeconds</td>
<td>Integer</td>
<td>Shows how long the supervisor is running in seconds</td>
</tr>
<tr>
<td>slotsTotal</td>
<td>Integer</td>
<td>Total number of available worker slots for this supervisor</td>
</tr>
<tr>
<td>slotsUsed</td>
<td>Integer</td>
<td>Number of worker slots used on this supervisor</td>
</tr>
<tr>
<td>totalMem</td>
<td>Double</td>
<td>Total memory capacity on this supervisor</td>
</tr>
<tr>
<td>totalCpu</td>
<td>Double</td>
<td>Total CPU capacity on this supervisor</td>
</tr>
<tr>
<td>usedMem</td>
<td>Double</td>
<td>Used memory capacity on this supervisor</td>
</tr>
<tr>
<td>usedCpu</td>
<td>Double</td>
<td>Used CPU capacity on this supervisor</td>
</tr>
</tbody></table>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"supervisors"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0b879808-2a26-442b-8f7d-23101e0c3696"</span><span class="p">,</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"5m 58s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">358</span><span class="p">,</span><span class="w">
</span><span class="s2">"slotsTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">4</span><span class="p">,</span><span class="w">
</span><span class="s2">"slotsUsed"</span><span class="p">:</span><span class="w"> </span><span class="mi">3</span><span class="p">,</span><span class="w">
</span><span class="s2">"totalMem"</span><span class="p">:</span><span class="w"> </span><span class="mi">3000</span><span class="p">,</span><span class="w">
</span><span class="s2">"totalCpu"</span><span class="p">:</span><span class="w"> </span><span class="mi">400</span><span class="p">,</span><span class="w">
</span><span class="s2">"usedMem"</span><span class="p">:</span><span class="w"> </span><span class="mi">1280</span><span class="p">,</span><span class="w">
</span><span class="s2">"usedCPU"</span><span class="p">:</span><span class="w"> </span><span class="mi">160</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"schedulerDisplayResource"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-nimbus-summary-get">/api/v1/nimbus/summary (GET)</h3>
<p>Returns summary information for all nimbus hosts.</p>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>host</td>
<td>String</td>
<td>Nimbus&#39; host name</td>
</tr>
<tr>
<td>port</td>
<td>int</td>
<td>Nimbus&#39; port number</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Possible values are Leader, Not a Leader, Dead</td>
</tr>
<tr>
<td>nimbusUpTime</td>
<td>String</td>
<td>Shows since how long the nimbus has been running</td>
</tr>
<tr>
<td>nimbusUpTimeSeconds</td>
<td>String</td>
<td>Shows since how long the nimbus has been running in seconds</td>
</tr>
<tr>
<td>nimbusLogLink</td>
<td>String</td>
<td>Logviewer url to view the nimbus.log</td>
</tr>
<tr>
<td>version</td>
<td>String</td>
<td>Version of storm this nimbus host is running</td>
</tr>
</tbody></table>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"nimbuses"</span><span class="p">:[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="s2">"192.168.202.1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="mi">6627</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbusLogLink"</span><span class="p">:</span><span class="s2">"http:</span><span class="se">\/\/</span><span class="s2">192.168.202.1:8000</span><span class="se">\/</span><span class="s2">log?file=nimbus.log"</span><span class="p">,</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="err">Leader</span><span class="p">,</span><span class="w">
</span><span class="s2">"version"</span><span class="p">:</span><span class="s2">"0.10.0-SNAPSHOT"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbusUpTime"</span><span class="p">:</span><span class="s2">"3m 33s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbusUpTimeSeconds"</span><span class="p">:</span><span class="s2">"213"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-history-summary-get">/api/v1/history/summary (GET)</h3>
<p>Returns a list of all running topologies&#39; IDs submitted by the current user.</p>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>topo-history</td>
<td>List</td>
<td>List of Topologies&#39; IDs</td>
</tr>
</tbody></table>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"topo-history"</span><span class="p">:[</span><span class="w">
</span><span class="s2">"wc6-1-1446571009"</span><span class="p">,</span><span class="w">
</span><span class="s2">"wc8-2-1446587178"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-summary-get">/api/v1/topology/summary (GET)</h3>
<p>Returns summary information for all topologies.</p>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Topology Id</td>
</tr>
<tr>
<td>name</td>
<td>String</td>
<td>Topology Name</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Topology Status</td>
</tr>
<tr>
<td>uptime</td>
<td>String</td>
<td>Shows how long the topology is running</td>
</tr>
<tr>
<td>uptimeSeconds</td>
<td>Integer</td>
<td>Shows how long the topology is running in seconds</td>
</tr>
<tr>
<td>tasksTotal</td>
<td>Integer</td>
<td>Total number of tasks for this topology</td>
</tr>
<tr>
<td>workersTotal</td>
<td>Integer</td>
<td>Number of workers used for this topology</td>
</tr>
<tr>
<td>executorsTotal</td>
<td>Integer</td>
<td>Number of executors used for this topology</td>
</tr>
<tr>
<td>replicationCount</td>
<td>Integer</td>
<td>Number of nimbus hosts on which this topology code is replicated</td>
</tr>
<tr>
<td>requestedMemOnHeap</td>
<td>Double</td>
<td>Requested On-Heap Memory by User (MB)</td>
</tr>
<tr>
<td>requestedMemOffHeap</td>
<td>Double</td>
<td>Requested Off-Heap Memory by User (MB)</td>
</tr>
<tr>
<td>requestedTotalMem</td>
<td>Double</td>
<td>Requested Total Memory by User (MB)</td>
</tr>
<tr>
<td>requestedCpu</td>
<td>Double</td>
<td>Requested CPU by User (%)</td>
</tr>
<tr>
<td>assignedMemOnHeap</td>
<td>Double</td>
<td>Assigned On-Heap Memory by Scheduler (MB)</td>
</tr>
<tr>
<td>assignedMemOffHeap</td>
<td>Double</td>
<td>Assigned Off-Heap Memory by Scheduler (MB)</td>
</tr>
<tr>
<td>assignedTotalMem</td>
<td>Double</td>
<td>Assigned Total Memory by Scheduler (MB)</td>
</tr>
<tr>
<td>assignedCpu</td>
<td>Double</td>
<td>Assigned CPU by Scheduler (%)</td>
</tr>
</tbody></table>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"topologies"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"WordCount3-1-1402960825"</span><span class="p">,</span><span class="w">
</span><span class="s2">"name"</span><span class="p">:</span><span class="w"> </span><span class="s2">"WordCount3"</span><span class="p">,</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ACTIVE"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"6m 5s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">365</span><span class="p">,</span><span class="w">
</span><span class="s2">"tasksTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">28</span><span class="p">,</span><span class="w">
</span><span class="s2">"workersTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">3</span><span class="p">,</span><span class="w">
</span><span class="s2">"executorsTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">28</span><span class="p">,</span><span class="w">
</span><span class="s2">"replicationCount"</span><span class="p">:</span><span class="w"> </span><span class="mi">1</span><span class="p">,</span><span class="w">
</span><span class="s2">"requestedMemOnHeap"</span><span class="p">:</span><span class="w"> </span><span class="mi">640</span><span class="p">,</span><span class="w">
</span><span class="s2">"requestedMemOffHeap"</span><span class="p">:</span><span class="w"> </span><span class="mi">128</span><span class="p">,</span><span class="w">
</span><span class="s2">"requestedTotalMem"</span><span class="p">:</span><span class="w"> </span><span class="mi">768</span><span class="p">,</span><span class="w">
</span><span class="s2">"requestedCpu"</span><span class="p">:</span><span class="w"> </span><span class="mi">80</span><span class="p">,</span><span class="w">
</span><span class="s2">"assignedMemOnHeap"</span><span class="p">:</span><span class="w"> </span><span class="mi">640</span><span class="p">,</span><span class="w">
</span><span class="s2">"assignedMemOffHeap"</span><span class="p">:</span><span class="w"> </span><span class="mi">128</span><span class="p">,</span><span class="w">
</span><span class="s2">"assignedTotalMem"</span><span class="p">:</span><span class="w"> </span><span class="mi">768</span><span class="p">,</span><span class="w">
</span><span class="s2">"assignedCpu"</span><span class="p">:</span><span class="w"> </span><span class="mi">80</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="s2">"schedulerDisplayResource"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-workers-id-get">/api/v1/topology-workers/:id (GET)</h3>
<p>Returns the worker&#39; information (host and port) for a topology.</p>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>hostPortList</td>
<td>List</td>
<td>Workers&#39; information for a topology</td>
</tr>
<tr>
<td>name</td>
<td>Integer</td>
<td>Logviewer Port</td>
</tr>
</tbody></table>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"hostPortList"</span><span class="p">:[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="s2">"192.168.202.2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="mi">6701</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="s2">"192.168.202.2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="mi">6702</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="s2">"192.168.202.3"</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="mi">6700</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"logviewerPort"</span><span class="p">:</span><span class="mi">8000</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-get">/api/v1/topology/:id (GET)</h3>
<p>Returns topology information and statistics. Substitute id with topology id.</p>
<p>Request parameters:</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>window</td>
<td>String. Default value :all-time</td>
<td>Window duration for metrics in seconds</td>
</tr>
<tr>
<td>sys</td>
<td>String. Values 1 or 0. Default value 0</td>
<td>Controls including sys stats part of the response</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Topology Id</td>
</tr>
<tr>
<td>name</td>
<td>String</td>
<td>Topology Name</td>
</tr>
<tr>
<td>uptime</td>
<td>String</td>
<td>How long the topology has been running</td>
</tr>
<tr>
<td>uptimeSeconds</td>
<td>Integer</td>
<td>How long the topology has been running in seconds</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Current status of the topology, e.g. &quot;ACTIVE&quot;</td>
</tr>
<tr>
<td>tasksTotal</td>
<td>Integer</td>
<td>Total number of tasks for this topology</td>
</tr>
<tr>
<td>workersTotal</td>
<td>Integer</td>
<td>Number of workers used for this topology</td>
</tr>
<tr>
<td>executorsTotal</td>
<td>Integer</td>
<td>Number of executors used for this topology</td>
</tr>
<tr>
<td>msgTimeout</td>
<td>Integer</td>
<td>Number of seconds a tuple has before the spout considers it failed</td>
</tr>
<tr>
<td>windowHint</td>
<td>String</td>
<td>window param value in &quot;hh mm ss&quot; format. Default value is &quot;All Time&quot;</td>
</tr>
<tr>
<td>schedulerDisplayResource</td>
<td>Boolean</td>
<td>Whether to display scheduler resource information</td>
</tr>
<tr>
<td>topologyStats</td>
<td>Array</td>
<td>Array of all the topology related stats per time window</td>
</tr>
<tr>
<td>topologyStats.windowPretty</td>
<td>String</td>
<td>Duration passed in HH:MM:SS format</td>
</tr>
<tr>
<td>topologyStats.window</td>
<td>String</td>
<td>User requested time window for metrics</td>
</tr>
<tr>
<td>topologyStats.emitted</td>
<td>Long</td>
<td>Number of messages emitted in given window</td>
</tr>
<tr>
<td>topologyStats.trasferred</td>
<td>Long</td>
<td>Number messages transferred in given window</td>
</tr>
<tr>
<td>topologyStats.completeLatency</td>
<td>String (double value returned in String format)</td>
<td>Total latency for processing the message</td>
</tr>
<tr>
<td>topologyStats.acked</td>
<td>Long</td>
<td>Number of messages acked in given window</td>
</tr>
<tr>
<td>topologyStats.failed</td>
<td>Long</td>
<td>Number of messages failed in given window</td>
</tr>
<tr>
<td>spouts</td>
<td>Array</td>
<td>Array of all the spout components in the topology</td>
</tr>
<tr>
<td>spouts.spoutId</td>
<td>String</td>
<td>Spout id</td>
</tr>
<tr>
<td>spouts.executors</td>
<td>Integer</td>
<td>Number of executors for the spout</td>
</tr>
<tr>
<td>spouts.emitted</td>
<td>Long</td>
<td>Number of messages emitted in given window</td>
</tr>
<tr>
<td>spouts.completeLatency</td>
<td>String (double value returned in String format)</td>
<td>Total latency for processing the message</td>
</tr>
<tr>
<td>spouts.transferred</td>
<td>Long</td>
<td>Total number of messages transferred in given window</td>
</tr>
<tr>
<td>spouts.tasks</td>
<td>Integer</td>
<td>Total number of tasks for the spout</td>
</tr>
<tr>
<td>spouts.lastError</td>
<td>String</td>
<td>Shows the last error happened in a spout</td>
</tr>
<tr>
<td>spouts.errorLapsedSecs</td>
<td>Integer</td>
<td>Number of seconds elapsed since that last error happened in a spout</td>
</tr>
<tr>
<td>spouts.errorWorkerLogLink</td>
<td>String</td>
<td>Link to the worker log that reported the exception</td>
</tr>
<tr>
<td>spouts.acked</td>
<td>Long</td>
<td>Number of messages acked</td>
</tr>
<tr>
<td>spouts.failed</td>
<td>Long</td>
<td>Number of messages failed</td>
</tr>
<tr>
<td>bolts</td>
<td>Array</td>
<td>Array of bolt components in the topology</td>
</tr>
<tr>
<td>bolts.boltId</td>
<td>String</td>
<td>Bolt id</td>
</tr>
<tr>
<td>bolts.capacity</td>
<td>String (double value returned in String format)</td>
<td>This value indicates number of messages executed * average execute latency / time window</td>
</tr>
<tr>
<td>bolts.processLatency</td>
<td>String (double value returned in String format)</td>
<td>Average time of the bolt to ack a message after it was received</td>
</tr>
<tr>
<td>bolts.executeLatency</td>
<td>String (double value returned in String format)</td>
<td>Average time to run the execute method of the bolt</td>
</tr>
<tr>
<td>bolts.executors</td>
<td>Integer</td>
<td>Number of executor tasks in the bolt component</td>
</tr>
<tr>
<td>bolts.tasks</td>
<td>Integer</td>
<td>Number of instances of bolt</td>
</tr>
<tr>
<td>bolts.acked</td>
<td>Long</td>
<td>Number of tuples acked by the bolt</td>
</tr>
<tr>
<td>bolts.failed</td>
<td>Long</td>
<td>Number of tuples failed by the bolt</td>
</tr>
<tr>
<td>bolts.lastError</td>
<td>String</td>
<td>Shows the last error occurred in the bolt</td>
</tr>
<tr>
<td>bolts.errorLapsedSecs</td>
<td>Integer</td>
<td>Number of seconds elapsed since that last error happened in a bolt</td>
</tr>
<tr>
<td>bolts.errorWorkerLogLink</td>
<td>String</td>
<td>Link to the worker log that reported the exception</td>
</tr>
<tr>
<td>bolts.emitted</td>
<td>Long</td>
<td>Number of tuples emitted</td>
</tr>
<tr>
<td>replicationCount</td>
<td>Integer</td>
<td>Number of nimbus hosts on which this topology code is replicated</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight"> 1. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825
2. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825?sys=1
3. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825?window=600
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"name"</span><span class="p">:</span><span class="w"> </span><span class="s2">"WordCount3"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"WordCount3-1-1402960825"</span><span class="p">,</span><span class="w">
</span><span class="s2">"workersTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">3</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"600"</span><span class="p">,</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ACTIVE"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tasksTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">28</span><span class="p">,</span><span class="w">
</span><span class="s2">"executorsTotal"</span><span class="p">:</span><span class="w"> </span><span class="mi">28</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"29m 19s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">1759</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgTimeout"</span><span class="p">:</span><span class="w"> </span><span class="mi">30</span><span class="p">,</span><span class="w">
</span><span class="s2">"windowHint"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"schedulerDisplayResource"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"topologyStats"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"600"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">397960</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">213380</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">213460</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"3h 0m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10800"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">1190260</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">638260</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">638280</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"1d 0h 0m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"86400"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">1190260</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">638260</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">638280</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"All time"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">":all-time"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">1190260</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">638260</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">638280</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"spouts"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"executors"</span><span class="p">:</span><span class="w"> </span><span class="mi">5</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">28880</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">28880</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"spoutId"</span><span class="p">:</span><span class="w"> </span><span class="s2">"spout"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tasks"</span><span class="p">:</span><span class="w"> </span><span class="mi">5</span><span class="p">,</span><span class="w">
</span><span class="s2">"lastError"</span><span class="p">:</span><span class="w"> </span><span class="s2">""</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorLapsedSecs"</span><span class="p">:</span><span class="w"> </span><span class="kc">null</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"bolts"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"executors"</span><span class="p">:</span><span class="w"> </span><span class="mi">12</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">184580</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">184640</span><span class="p">,</span><span class="w">
</span><span class="s2">"executeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.048"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tasks"</span><span class="p">:</span><span class="w"> </span><span class="mi">12</span><span class="p">,</span><span class="w">
</span><span class="s2">"executed"</span><span class="p">:</span><span class="w"> </span><span class="mi">184620</span><span class="p">,</span><span class="w">
</span><span class="s2">"processLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.043"</span><span class="p">,</span><span class="w">
</span><span class="s2">"boltId"</span><span class="p">:</span><span class="w"> </span><span class="s2">"count"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lastError"</span><span class="p">:</span><span class="w"> </span><span class="s2">""</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorLapsedSecs"</span><span class="p">:</span><span class="w"> </span><span class="kc">null</span><span class="p">,</span><span class="w">
</span><span class="s2">"capacity"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.003"</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"executors"</span><span class="p">:</span><span class="w"> </span><span class="mi">8</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">184500</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">184500</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">28820</span><span class="p">,</span><span class="w">
</span><span class="s2">"executeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.024"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tasks"</span><span class="p">:</span><span class="w"> </span><span class="mi">8</span><span class="p">,</span><span class="w">
</span><span class="s2">"executed"</span><span class="p">:</span><span class="w"> </span><span class="mi">28780</span><span class="p">,</span><span class="w">
</span><span class="s2">"processLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"2.112"</span><span class="p">,</span><span class="w">
</span><span class="s2">"boltId"</span><span class="p">:</span><span class="w"> </span><span class="s2">"split"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lastError"</span><span class="p">:</span><span class="w"> </span><span class="s2">""</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorLapsedSecs"</span><span class="p">:</span><span class="w"> </span><span class="kc">null</span><span class="p">,</span><span class="w">
</span><span class="s2">"capacity"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"configuration"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"storm.id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"WordCount3-1-1402960825"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dev.zookeeper.path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/tmp/dev-storm-zookeeper"</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.tick.tuple.freq.secs"</span><span class="p">:</span><span class="w"> </span><span class="kc">null</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.builtin.metrics.bucket.size.secs"</span><span class="p">:</span><span class="w"> </span><span class="mi">60</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.fall.back.on.java.serialization"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.max.error.report.per.interval"</span><span class="p">:</span><span class="w"> </span><span class="mi">5</span><span class="p">,</span><span class="w">
</span><span class="s2">"zmq.linger.millis"</span><span class="p">:</span><span class="w"> </span><span class="mi">5000</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.skip.missing.kryo.registrations"</span><span class="p">:</span><span class="w"> </span><span class="kc">false</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.messaging.netty.client_worker_threads"</span><span class="p">:</span><span class="w"> </span><span class="mi">1</span><span class="p">,</span><span class="w">
</span><span class="s2">"ui.childopts"</span><span class="p">:</span><span class="w"> </span><span class="s2">"-Xmx768m"</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.zookeeper.session.timeout"</span><span class="p">:</span><span class="w"> </span><span class="mi">20000</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbus.reassign"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.trident.batch.emit.interval.millis"</span><span class="p">:</span><span class="w"> </span><span class="mi">500</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.messaging.netty.flush.check.interval.ms"</span><span class="p">:</span><span class="w"> </span><span class="mi">10</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbus.monitor.freq.secs"</span><span class="p">:</span><span class="w"> </span><span class="mi">10</span><span class="p">,</span><span class="w">
</span><span class="s2">"logviewer.childopts"</span><span class="p">:</span><span class="w"> </span><span class="s2">"-Xmx128m"</span><span class="p">,</span><span class="w">
</span><span class="s2">"java.library.path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/usr/local/lib:/opt/local/lib:/usr/lib"</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.executor.send.buffer.size"</span><span class="p">:</span><span class="w"> </span><span class="mi">1024</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.local.dir"</span><span class="p">:</span><span class="w"> </span><span class="s2">"storm-local"</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.messaging.netty.buffer_size"</span><span class="p">:</span><span class="w"> </span><span class="mi">5242880</span><span class="p">,</span><span class="w">
</span><span class="s2">"supervisor.worker.start.timeout.secs"</span><span class="p">:</span><span class="w"> </span><span class="mi">120</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.enable.message.timeouts"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbus.cleanup.inbox.freq.secs"</span><span class="p">:</span><span class="w"> </span><span class="mi">600</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbus.inbox.jar.expiration.secs"</span><span class="p">:</span><span class="w"> </span><span class="mi">3600</span><span class="p">,</span><span class="w">
</span><span class="s2">"drpc.worker.threads"</span><span class="p">:</span><span class="w"> </span><span class="mi">64</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.worker.shared.thread.pool.size"</span><span class="p">:</span><span class="w"> </span><span class="mi">4</span><span class="p">,</span><span class="w">
</span><span class="s2">"nimbus.seeds"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"hw10843.local"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"storm.messaging.netty.min_wait_ms"</span><span class="p">:</span><span class="w"> </span><span class="mi">100</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.zookeeper.port"</span><span class="p">:</span><span class="w"> </span><span class="mi">2181</span><span class="p">,</span><span class="w">
</span><span class="s2">"transactional.zookeeper.port"</span><span class="p">:</span><span class="w"> </span><span class="kc">null</span><span class="p">,</span><span class="w">
</span><span class="s2">"topology.executor.receive.buffer.size"</span><span class="p">:</span><span class="w"> </span><span class="mi">1024</span><span class="p">,</span><span class="w">
</span><span class="s2">"transactional.zookeeper.servers"</span><span class="p">:</span><span class="w"> </span><span class="kc">null</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.zookeeper.root"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/storm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.zookeeper.retry.intervalceiling.millis"</span><span class="p">:</span><span class="w"> </span><span class="mi">30000</span><span class="p">,</span><span class="w">
</span><span class="s2">"supervisor.enable"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"storm.messaging.netty.server_worker_threads"</span><span class="p">:</span><span class="w"> </span><span class="mi">1</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="s2">"replicationCount"</span><span class="p">:</span><span class="w"> </span><span class="mi">1</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-component-component-get">/api/v1/topology/:id/component/:component (GET)</h3>
<p>Returns detailed metrics and executor information</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>component</td>
<td>String (required)</td>
<td>Component Id</td>
</tr>
<tr>
<td>window</td>
<td>String. Default value :all-time</td>
<td>window duration for metrics in seconds</td>
</tr>
<tr>
<td>sys</td>
<td>String. Values 1 or 0. Default value 0</td>
<td>controls including sys stats part of the response</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Component id</td>
</tr>
<tr>
<td>name</td>
<td>String</td>
<td>Topology name</td>
</tr>
<tr>
<td>componentType</td>
<td>String</td>
<td>component type: SPOUT or BOLT</td>
</tr>
<tr>
<td>windowHint</td>
<td>String</td>
<td>window param value in &quot;hh mm ss&quot; format. Default value is &quot;All Time&quot;</td>
</tr>
<tr>
<td>executors</td>
<td>Integer</td>
<td>Number of executor tasks in the component</td>
</tr>
<tr>
<td>componentErrors</td>
<td>Array of Errors</td>
<td>List of component errors</td>
</tr>
<tr>
<td>componentErrors.errorTime</td>
<td>Long</td>
<td>Timestamp when the exception occurred (Prior to 0.11.0, this field was named &#39;time&#39;.)</td>
</tr>
<tr>
<td>componentErrors.errorHost</td>
<td>String</td>
<td>host name for the error</td>
</tr>
<tr>
<td>componentErrors.errorPort</td>
<td>String</td>
<td>port for the error</td>
</tr>
<tr>
<td>componentErrors.error</td>
<td>String</td>
<td>Shows the error happened in a component</td>
</tr>
<tr>
<td>componentErrors.errorLapsedSecs</td>
<td>Integer</td>
<td>Number of seconds elapsed since the error happened in a component</td>
</tr>
<tr>
<td>componentErrors.errorWorkerLogLink</td>
<td>String</td>
<td>Link to the worker log that reported the exception</td>
</tr>
<tr>
<td>topologyId</td>
<td>String</td>
<td>Topology id</td>
</tr>
<tr>
<td>tasks</td>
<td>Integer</td>
<td>Number of instances of component</td>
</tr>
<tr>
<td>window</td>
<td>String. Default value &quot;All Time&quot;</td>
<td>window duration for metrics in seconds</td>
</tr>
<tr>
<td>spoutSummary or boltStats</td>
<td>Array</td>
<td>Array of component stats. <strong>Please note this element tag can be spoutSummary or boltStats depending on the componentType</strong></td>
</tr>
<tr>
<td>spoutSummary.windowPretty</td>
<td>String</td>
<td>Duration passed in HH:MM:SS format</td>
</tr>
<tr>
<td>spoutSummary.window</td>
<td>String</td>
<td>window duration for metrics in seconds</td>
</tr>
<tr>
<td>spoutSummary.emitted</td>
<td>Long</td>
<td>Number of messages emitted in given window</td>
</tr>
<tr>
<td>spoutSummary.completeLatency</td>
<td>String (double value returned in String format)</td>
<td>Total latency for processing the message</td>
</tr>
<tr>
<td>spoutSummary.transferred</td>
<td>Long</td>
<td>Total number of messages transferred in given window</td>
</tr>
<tr>
<td>spoutSummary.acked</td>
<td>Long</td>
<td>Number of messages acked</td>
</tr>
<tr>
<td>spoutSummary.failed</td>
<td>Long</td>
<td>Number of messages failed</td>
</tr>
<tr>
<td>boltStats.windowPretty</td>
<td>String</td>
<td>Duration passed in HH:MM:SS format</td>
</tr>
<tr>
<td>boltStats..window</td>
<td>String</td>
<td>window duration for metrics in seconds</td>
</tr>
<tr>
<td>boltStats.transferred</td>
<td>Long</td>
<td>Total number of messages transferred in given window</td>
</tr>
<tr>
<td>boltStats.processLatency</td>
<td>String (double value returned in String format)</td>
<td>Average time of the bolt to ack a message after it was received</td>
</tr>
<tr>
<td>boltStats.acked</td>
<td>Long</td>
<td>Number of messages acked</td>
</tr>
<tr>
<td>boltStats.failed</td>
<td>Long</td>
<td>Number of messages failed</td>
</tr>
<tr>
<td>profilingAndDebuggingCapable</td>
<td>Boolean</td>
<td>true if there is support for Profiling and Debugging Actions</td>
</tr>
<tr>
<td>profileActionEnabled</td>
<td>Boolean</td>
<td>true if worker profiling (Java Flight Recorder) is enabled</td>
</tr>
<tr>
<td>profilerActive</td>
<td>Array</td>
<td>Array of currently active Profiler Actions</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">1. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout
2. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout?sys=1
3. http://ui-daemon-host-name:8080/api/v1/topology/WordCount3-1-1402960825/component/spout?window=600
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"name"</span><span class="p">:</span><span class="w"> </span><span class="s2">"WordCount3"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"spout"</span><span class="p">,</span><span class="w">
</span><span class="s2">"componentType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"spout"</span><span class="p">,</span><span class="w">
</span><span class="s2">"windowHint"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"executors"</span><span class="p">:</span><span class="w"> </span><span class="mi">5</span><span class="p">,</span><span class="w">
</span><span class="s2">"componentErrors"</span><span class="p">:[{</span><span class="s2">"errorTime"</span><span class="p">:</span><span class="w"> </span><span class="mi">1406006074000</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorHost"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.70"</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorPort"</span><span class="p">:</span><span class="w"> </span><span class="mi">6701</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorWorkerLogLink"</span><span class="p">:</span><span class="w"> </span><span class="s2">"http://10.11.1.7:8000/log?file=worker-6701.log"</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorLapsedSecs"</span><span class="p">:</span><span class="w"> </span><span class="mi">16</span><span class="p">,</span><span class="w">
</span><span class="s2">"error"</span><span class="p">:</span><span class="w"> </span><span class="s2">"java.lang.RuntimeException: java.lang.StringIndexOutOfBoundsException: Some Error</span><span class="se">\n\t</span><span class="s2">at org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:128)</span><span class="se">\n\t</span><span class="s2">at org.apache.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:99)</span><span class="se">\n\t</span><span class="s2">at org.apache.storm.disruptor$consume_batch_when_available.invoke(disruptor.clj:80)</span><span class="se">\n\t</span><span class="s2">at backtype...more.."</span><span class="w">
</span><span class="p">}],</span><span class="w">
</span><span class="s2">"topologyId"</span><span class="p">:</span><span class="w"> </span><span class="s2">"WordCount3-1-1402960825"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tasks"</span><span class="p">:</span><span class="w"> </span><span class="mi">5</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"600"</span><span class="p">,</span><span class="w">
</span><span class="s2">"profilerActive"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.70"</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="w"> </span><span class="s2">"6701"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dumplink"</span><span class="p">:</span><span class="s2">"http:</span><span class="se">\/\/</span><span class="s2">10.11.1.70:8000</span><span class="se">\/</span><span class="s2">dumps</span><span class="se">\/</span><span class="s2">ex-1-1452718803</span><span class="se">\/</span><span class="s2">10.11.1.70%3A6701"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timestamp"</span><span class="p">:</span><span class="s2">"576328"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"profilingAndDebuggingCapable"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"profileActionEnabled"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"spoutSummary"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"600"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">28500</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">28460</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"3h 0m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10800"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">127640</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">127440</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"1d 0h 0m 0s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">"86400"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">127640</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">127440</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"windowPretty"</span><span class="p">:</span><span class="w"> </span><span class="s2">"All time"</span><span class="p">,</span><span class="w">
</span><span class="s2">"window"</span><span class="p">:</span><span class="w"> </span><span class="s2">":all-time"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">127640</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">127440</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"outputStats"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"stream"</span><span class="p">:</span><span class="w"> </span><span class="s2">"__metrics"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">40</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"stream"</span><span class="p">:</span><span class="w"> </span><span class="s2">"default"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">28460</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">28460</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"executorStats"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"workerLogLink"</span><span class="p">:</span><span class="w"> </span><span class="s2">"http://10.11.1.7:8000/log?file=worker-6701.log"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">5720</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="w"> </span><span class="mi">6701</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">5720</span><span class="p">,</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"43m 4s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">2584</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"[24-24]"</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"workerLogLink"</span><span class="p">:</span><span class="w"> </span><span class="s2">"http://10.11.1.7:8000/log?file=worker-6703.log"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">5700</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="w"> </span><span class="mi">6703</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">5700</span><span class="p">,</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"42m 57s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">2577</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"[25-25]"</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"workerLogLink"</span><span class="p">:</span><span class="w"> </span><span class="s2">"http://10.11.1.7:8000/log?file=worker-6702.log"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">5700</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="w"> </span><span class="mi">6702</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">5680</span><span class="p">,</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"42m 57s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">2577</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"[26-26]"</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"workerLogLink"</span><span class="p">:</span><span class="w"> </span><span class="s2">"http://10.11.1.7:8000/log?file=worker-6701.log"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">5700</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="w"> </span><span class="mi">6701</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">5680</span><span class="p">,</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"43m 4s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">2584</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"[27-27]"</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"workerLogLink"</span><span class="p">:</span><span class="w"> </span><span class="s2">"http://10.11.1.7:8000/log?file=worker-6703.log"</span><span class="p">,</span><span class="w">
</span><span class="s2">"emitted"</span><span class="p">:</span><span class="w"> </span><span class="mi">5680</span><span class="p">,</span><span class="w">
</span><span class="s2">"port"</span><span class="p">:</span><span class="w"> </span><span class="mi">6703</span><span class="p">,</span><span class="w">
</span><span class="s2">"completeLatency"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.000"</span><span class="p">,</span><span class="w">
</span><span class="s2">"transferred"</span><span class="p">:</span><span class="w"> </span><span class="mi">5680</span><span class="p">,</span><span class="w">
</span><span class="s2">"host"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7"</span><span class="p">,</span><span class="w">
</span><span class="s2">"acked"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptime"</span><span class="p">:</span><span class="w"> </span><span class="s2">"42m 57s"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uptimeSeconds"</span><span class="p">:</span><span class="w"> </span><span class="mi">2577</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"[28-28]"</span><span class="p">,</span><span class="w">
</span><span class="s2">"failed"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h2 id="profiling-and-debugging-get-operations">Profiling and Debugging GET Operations</h2>
<h3 id="api-v1-topology-id-profiling-start-host-port-timeout-get">/api/v1/topology/:id/profiling/start/:host-port/:timeout (GET)</h3>
<p>Request to start profiler on worker with timeout. Returns status and link to profiler artifacts for worker.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>host-port</td>
<td>String (required)</td>
<td>Worker Id</td>
</tr>
<tr>
<td>timeout</td>
<td>String (required)</td>
<td>Time out for profiler to stop in minutes</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Worker id</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Response Status</td>
</tr>
<tr>
<td>timeout</td>
<td>String</td>
<td>Requested timeout</td>
</tr>
<tr>
<td>dumplink</td>
<td>String</td>
<td>Link to logviewer URL for worker profiler documents.</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/10
2. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/5
3. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/start/10.11.1.7:6701/20
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ok"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7:6701"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timeout"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dumplink"</span><span class="p">:</span><span class="w"> </span><span class="s2">"http:</span><span class="se">\/\/</span><span class="s2">10.11.1.7:8000</span><span class="se">\/</span><span class="s2">dumps</span><span class="se">\/</span><span class="s2">wordcount-1-1446614150</span><span class="se">\/</span><span class="s2">10.11.1.7%3A6701"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-profiling-dumpprofile-host-port-get">/api/v1/topology/:id/profiling/dumpprofile/:host-port (GET)</h3>
<p>Request to dump profiler recording on worker. Returns status and worker id for the request.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>host-port</td>
<td>String (required)</td>
<td>Worker Id</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Worker id</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Response Status</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpprofile/10.11.1.7:6701
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ok"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7:6701"</span><span class="p">,</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-profiling-stop-host-port-get">/api/v1/topology/:id/profiling/stop/:host-port (GET)</h3>
<p>Request to stop profiler on worker. Returns status and worker id for the request.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>host-port</td>
<td>String (required)</td>
<td>Worker Id</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Worker id</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Response Status</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/stop/10.11.1.7:6701
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ok"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7:6701"</span><span class="p">,</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-profiling-dumpjstack-host-port-get">/api/v1/topology/:id/profiling/dumpjstack/:host-port (GET)</h3>
<p>Request to dump jstack on worker. Returns status and worker id for the request.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>host-port</td>
<td>String (required)</td>
<td>Worker Id</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Worker id</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Response Status</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpjstack/10.11.1.7:6701
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ok"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7:6701"</span><span class="p">,</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-profiling-dumpheap-host-port-get">/api/v1/topology/:id/profiling/dumpheap/:host-port (GET)</h3>
<p>Request to dump heap (jmap) on worker. Returns status and worker id for the request.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>host-port</td>
<td>String (required)</td>
<td>Worker Id</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Worker id</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Response Status</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/dumpheap/10.11.1.7:6701
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ok"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7:6701"</span><span class="p">,</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-profiling-restartworker-host-port-get">/api/v1/topology/:id/profiling/restartworker/:host-port (GET)</h3>
<p>Request to request the worker. Returns status and worker id for the request.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>host-port</td>
<td>String (required)</td>
<td>Worker Id</td>
</tr>
</tbody></table>
<p>Response fields:</p>
<table><thead>
<tr>
<th>Field</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String</td>
<td>Worker id</td>
</tr>
<tr>
<td>status</td>
<td>String</td>
<td>Response Status</td>
</tr>
</tbody></table>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">1. http://ui-daemon-host-name:8080/api/v1/topology/wordcount-1-1446614150/profiling/restartworker/10.11.1.7:6701
</code></pre></div>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"status"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ok"</span><span class="p">,</span><span class="w">
</span><span class="s2">"id"</span><span class="p">:</span><span class="w"> </span><span class="s2">"10.11.1.7:6701"</span><span class="p">,</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h2 id="post-operations">POST Operations</h2>
<h3 id="api-v1-topology-id-activate-post">/api/v1/topology/:id/activate (POST)</h3>
<p>Activates a topology.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
</tbody></table>
<p>Sample Response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="s2">"topologyOperation"</span><span class="p">:</span><span class="s2">"activate"</span><span class="p">,</span><span class="s2">"topologyId"</span><span class="p">:</span><span class="s2">"wordcount-1-1420308665"</span><span class="p">,</span><span class="s2">"status"</span><span class="p">:</span><span class="s2">"success"</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-deactivate-post">/api/v1/topology/:id/deactivate (POST)</h3>
<p>Deactivates a topology.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
</tbody></table>
<p>Sample Response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="s2">"topologyOperation"</span><span class="p">:</span><span class="s2">"deactivate"</span><span class="p">,</span><span class="s2">"topologyId"</span><span class="p">:</span><span class="s2">"wordcount-1-1420308665"</span><span class="p">,</span><span class="s2">"status"</span><span class="p">:</span><span class="s2">"success"</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-rebalance-wait-time-post">/api/v1/topology/:id/rebalance/:wait-time (POST)</h3>
<p>Rebalances a topology.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>wait-time</td>
<td>String (required)</td>
<td>Wait time before rebalance happens</td>
</tr>
<tr>
<td>rebalanceOptions</td>
<td>Json (optional)</td>
<td>topology rebalance options</td>
</tr>
</tbody></table>
<p>Sample rebalanceOptions json:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="s2">"rebalanceOptions"</span><span class="w"> </span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="s2">"numWorkers"</span><span class="w"> </span><span class="p">:</span><span class="w"> </span><span class="mi">2</span><span class="p">,</span><span class="w"> </span><span class="s2">"executors"</span><span class="w"> </span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="s2">"spout"</span><span class="w"> </span><span class="p">:</span><span class="mi">4</span><span class="p">,</span><span class="w"> </span><span class="s2">"count"</span><span class="w"> </span><span class="p">:</span><span class="w"> </span><span class="mi">10</span><span class="p">}},</span><span class="w"> </span><span class="s2">"callback"</span><span class="w"> </span><span class="p">:</span><span class="w"> </span><span class="s2">"foo"</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<p>Examples:</p>
<div class="highlight"><pre><code class="language-no-highlight" data-lang="no-highlight">curl -i -b ~/cookiejar.txt -c ~/cookiejar.txt -X POST
-H "Content-Type: application/json"
-d '{"rebalanceOptions": {"numWorkers": 2, "executors": { "spout" : "5", "split": 7, "count": 5 }}, "callback":"foo"}'
http://localhost:8080/api/v1/topology/wordcount-1-1420308665/rebalance/0
</code></pre></div>
<p>Sample Response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="s2">"topologyOperation"</span><span class="p">:</span><span class="s2">"rebalance"</span><span class="p">,</span><span class="s2">"topologyId"</span><span class="p">:</span><span class="s2">"wordcount-1-1420308665"</span><span class="p">,</span><span class="s2">"status"</span><span class="p">:</span><span class="s2">"success"</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h3 id="api-v1-topology-id-kill-wait-time-post">/api/v1/topology/:id/kill/:wait-time (POST)</h3>
<p>Kills a topology.</p>
<table><thead>
<tr>
<th>Parameter</th>
<th>Value</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td>id</td>
<td>String (required)</td>
<td>Topology Id</td>
</tr>
<tr>
<td>wait-time</td>
<td>String (required)</td>
<td>Wait time before rebalance happens</td>
</tr>
</tbody></table>
<p>Caution: Small wait times (0-5 seconds) may increase the probability of triggering the bug reported in
<a href="https://issues.apache.org/jira/browse/STORM-112">STORM-112</a>, which may result in broker Supervisor
daemons.</p>
<p>Sample Response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="s2">"topologyOperation"</span><span class="p">:</span><span class="s2">"kill"</span><span class="p">,</span><span class="s2">"topologyId"</span><span class="p">:</span><span class="s2">"wordcount-1-1420308665"</span><span class="p">,</span><span class="s2">"status"</span><span class="p">:</span><span class="s2">"success"</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h2 id="api-errors">API errors</h2>
<p>The API returns 500 HTTP status codes in case of any errors.</p>
<p>Sample response:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"error"</span><span class="p">:</span><span class="w"> </span><span class="s2">"Internal Server Error"</span><span class="p">,</span><span class="w">
</span><span class="s2">"errorMessage"</span><span class="p">:</span><span class="w"> </span><span class="s2">"java.lang.NullPointerException</span><span class="se">\n\t</span><span class="s2">at clojure.core$name.invoke(core.clj:1505)</span><span class="se">\n\t</span><span class="s2">at org.apache.storm.ui.core$component_page.invoke(core.clj:752)</span><span class="se">\n\t</span><span class="s2">at org.apache.storm.ui.core$fn__7766.invoke(core.clj:782)</span><span class="se">\n\t</span><span class="s2">at compojure.core$make_route$fn__5755.invoke(core.clj:93)</span><span class="se">\n\t</span><span class="s2">at compojure.core$if_route$fn__5743.invoke(core.clj:39)</span><span class="se">\n\t</span><span class="s2">at compojure.core$if_method$fn__5736.invoke(core.clj:24)</span><span class="se">\n\t</span><span class="s2">at compojure.core$routing$fn__5761.invoke(core.clj:106)</span><span class="se">\n\t</span><span class="s2">at clojure.core$some.invoke(core.clj:2443)</span><span class="se">\n\t</span><span class="s2">at compojure.core$routing.doInvoke(core.clj:106)</span><span class="se">\n\t</span><span class="s2">at clojure.lang.RestFn.applyTo(RestFn.java:139)</span><span class="se">\n\t</span><span class="s2">at clojure.core$apply.invoke(core.clj:619)</span><span class="se">\n\t</span><span class="s2">at compojure.core$routes$fn__5765.invoke(core.clj:111)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.reload$wrap_reload$fn__6880.invoke(reload.clj:14)</span><span class="se">\n\t</span><span class="s2">at org.apache.storm.ui.core$catch_errors$fn__7800.invoke(core.clj:836)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.keyword_params$wrap_keyword_params$fn__6319.invoke(keyword_params.clj:27)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.nested_params$wrap_nested_params$fn__6358.invoke(nested_params.clj:65)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.params$wrap_params$fn__6291.invoke(params.clj:55)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.multipart_params$wrap_multipart_params$fn__6386.invoke(multipart_params.clj:103)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.flash$wrap_flash$fn__6675.invoke(flash.clj:14)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.session$wrap_session$fn__6664.invoke(session.clj:43)</span><span class="se">\n\t</span><span class="s2">at ring.middleware.cookies$wrap_cookies$fn__6595.invoke(cookies.clj:160)</span><span class="se">\n\t</span><span class="s2">at ring.adapter.jetty$proxy_handler$fn__6112.invoke(jetty.clj:16)</span><span class="se">\n\t</span><span class="s2">at ring.adapter.jetty.proxy$org.mortbay.jetty.handler.AbstractHandler$0.handle(Unknown Source)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.handler.HandlerWrapper.handle(HandlerWrapper.java:152)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.Server.handle(Server.java:326)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.HttpConnection.handleRequest(HttpConnection.java:542)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.HttpConnection$RequestHandler.headerComplete(HttpConnection.java:928)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.HttpParser.parseNext(HttpParser.java:549)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.HttpParser.parseAvailable(HttpParser.java:212)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.HttpConnection.handle(HttpConnection.java:404)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.jetty.bio.SocketConnector$Connection.run(SocketConnector.java:228)</span><span class="se">\n\t</span><span class="s2">at org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)</span><span class="se">\n</span><span class="s2">"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div></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 Storm</h5>
<p>Storm integrates with any queueing system and any database system. Storm's spout abstraction makes it easy to integrate a new queuing system. Likewise, integrating 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 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 © 2015 <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>