blob: bb91679d46a137ff723ed00fcdd22aa7c03bc160 [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>Tutorial</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.2.3</h1>
</div>
<div class="col-md-2">
<a href="/downloads.html" class="btn-std btn-block btn-download">Download</a>
</div>
</div>
</div>
</header>
<!--Header End-->
<!--Navigation Begin-->
<div class="navbar" role="banner">
<div class="container-fluid">
<div class="navbar-header">
<button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bs-navbar-collapse">
<span class="icon-bar"></span>
<span class="icon-bar"></span>
<span class="icon-bar"></span>
</button>
</div>
<nav class="collapse navbar-collapse bs-navbar-collapse" role="navigation">
<ul class="nav navbar-nav">
<li><a href="/index.html" id="home">Home</a></li>
<li><a href="/getting-help.html" id="getting-help">Getting Help</a></li>
<li><a href="/about/integrates.html" id="project-info">Project Information</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="documentation">Documentation <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/releases/2.3.0/index.html">2.3.0</a></li>
<li><a href="/releases/2.2.0/index.html">2.2.0</a></li>
<li><a href="/releases/2.1.0/index.html">2.1.0</a></li>
<li><a href="/releases/2.0.0/index.html">2.0.0</a></li>
<li><a href="/releases/1.2.3/index.html">1.2.3</a></li>
</ul>
</li>
<li><a href="/talksAndVideos.html">Talks and Slideshows</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="contribute">Community <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/contribute/Contributing-to-Storm.html">Contributing</a></li>
<li><a href="/contribute/People.html">People</a></li>
<li><a href="/contribute/BYLAWS.html">ByLaws</a></li>
</ul>
</li>
<li><a href="/2021/09/27/storm230-released.html" id="news">News</a></li>
</ul>
</nav>
</div>
</div>
<div class="container-fluid">
<h1 class="page-title">Tutorial</h1>
<div class="row">
<div class="col-md-12">
<!-- Documentation -->
<p class="post-meta"></p>
<div class="documentation-content"><p>In this tutorial, you&#39;ll learn how to create Storm topologies and deploy them to a Storm cluster. Java will be the main language used, but a few examples will use Python to illustrate Storm&#39;s multi-language capabilities.</p>
<h2 id="preliminaries">Preliminaries</h2>
<p>This tutorial uses examples from the <a href="http://github.com/apache/storm/blob/v1.2.3/examples/storm-starter">storm-starter</a> project. It&#39;s recommended that you clone the project and follow along with the examples. Read <a href="Setting-up-development-environment.html">Setting up a development environment</a> and <a href="Creating-a-new-Storm-project.html">Creating a new Storm project</a> to get your machine set up.</p>
<h2 id="components-of-a-storm-cluster">Components of a Storm cluster</h2>
<p>A Storm cluster is superficially similar to a Hadoop cluster. Whereas on Hadoop you run &quot;MapReduce jobs&quot;, on Storm you run &quot;topologies&quot;. &quot;Jobs&quot; and &quot;topologies&quot; themselves are very different -- one key difference is that a MapReduce job eventually finishes, whereas a topology processes messages forever (or until you kill it).</p>
<p>There are two kinds of nodes on a Storm cluster: the master node and the worker nodes. The master node runs a daemon called &quot;Nimbus&quot; that is similar to Hadoop&#39;s &quot;JobTracker&quot;. Nimbus is responsible for distributing code around the cluster, assigning tasks to machines, and monitoring for failures.</p>
<p>Each worker node runs a daemon called the &quot;Supervisor&quot;. The supervisor listens for work assigned to its machine and starts and stops worker processes as necessary based on what Nimbus has assigned to it. Each worker process executes a subset of a topology; a running topology consists of many worker processes spread across many machines.</p>
<p><img src="images/storm-cluster.png" alt="Storm cluster"></p>
<p>All coordination between Nimbus and the Supervisors is done through a <a href="http://zookeeper.apache.org/">Zookeeper</a> cluster. Additionally, the Nimbus daemon and Supervisor daemons are fail-fast and stateless; all state is kept in Zookeeper or on local disk. This means you can kill -9 Nimbus or the Supervisors and they&#39;ll start back up like nothing happened. This design leads to Storm clusters being incredibly stable.</p>
<h2 id="topologies">Topologies</h2>
<p>To do realtime computation on Storm, you create what are called &quot;topologies&quot;. A topology is a graph of computation. Each node in a topology contains processing logic, and links between nodes indicate how data should be passed around between nodes.</p>
<p>Running a topology is straightforward. First, you package all your code and dependencies into a single jar. Then, you run a command like the following:</p>
<div class="highlight"><pre><code class="language-" data-lang="">storm jar all-my-code.jar org.apache.storm.MyTopology arg1 arg2
</code></pre></div>
<p>This runs the class <code>org.apache.storm.MyTopology</code> with the arguments <code>arg1</code> and <code>arg2</code>. The main function of the class defines the topology and submits it to Nimbus. The <code>storm jar</code> part takes care of connecting to Nimbus and uploading the jar.</p>
<p>Since topology definitions are just Thrift structs, and Nimbus is a Thrift service, you can create and submit topologies using any programming language. The above example is the easiest way to do it from a JVM-based language. See <a href="Running-topologies-on-a-production-cluster.html">Running topologies on a production cluster</a>] for more information on starting and stopping topologies.</p>
<h2 id="streams">Streams</h2>
<p>The core abstraction in Storm is the &quot;stream&quot;. A stream is an unbounded sequence of tuples. Storm provides the primitives for transforming a stream into a new stream in a distributed and reliable way. For example, you may transform a stream of tweets into a stream of trending topics.</p>
<p>The basic primitives Storm provides for doing stream transformations are &quot;spouts&quot; and &quot;bolts&quot;. Spouts and bolts have interfaces that you implement to run your application-specific logic.</p>
<p>A spout is a source of streams. For example, a spout may read tuples off of a <a href="http://github.com/nathanmarz/storm-kestrel">Kestrel</a> queue and emit them as a stream. Or a spout may connect to the Twitter API and emit a stream of tweets.</p>
<p>A bolt consumes any number of input streams, does some processing, and possibly emits new streams. Complex stream transformations, like computing a stream of trending topics from a stream of tweets, require multiple steps and thus multiple bolts. Bolts can do anything from run functions, filter tuples, do streaming aggregations, do streaming joins, talk to databases, and more.</p>
<p>Networks of spouts and bolts are packaged into a &quot;topology&quot; which is the top-level abstraction that you submit to Storm clusters for execution. A topology is a graph of stream transformations where each node is a spout or bolt. Edges in the graph indicate which bolts are subscribing to which streams. When a spout or bolt emits a tuple to a stream, it sends the tuple to every bolt that subscribed to that stream.</p>
<p><img src="images/topology.png" alt="A Storm topology"></p>
<p>Links between nodes in your topology indicate how tuples should be passed around. For example, if there is a link between Spout A and Bolt B, a link from Spout A to Bolt C, and a link from Bolt B to Bolt C, then everytime Spout A emits a tuple, it will send the tuple to both Bolt B and Bolt C. All of Bolt B&#39;s output tuples will go to Bolt C as well.</p>
<p>Each node in a Storm topology executes in parallel. In your topology, you can specify how much parallelism you want for each node, and then Storm will spawn that number of threads across the cluster to do the execution.</p>
<p>A topology runs forever, or until you kill it. Storm will automatically reassign any failed tasks. Additionally, Storm guarantees that there will be no data loss, even if machines go down and messages are dropped.</p>
<h2 id="data-model">Data model</h2>
<p>Storm uses tuples as its data model. A tuple is a named list of values, and a field in a tuple can be an object of any type. Out of the box, Storm supports all the primitive types, strings, and byte arrays as tuple field values. To use an object of another type, you just need to implement <a href="Serialization.html">a serializer</a> for the type.</p>
<p>Every node in a topology must declare the output fields for the tuples it emits. For example, this bolt declares that it emits 2-tuples with the fields &quot;double&quot; and &quot;triple&quot;:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">class</span> <span class="nc">DoubleAndTripleBolt</span> <span class="kd">extends</span> <span class="n">BaseRichBolt</span> <span class="o">{</span>
<span class="kd">private</span> <span class="n">OutputCollectorBase</span> <span class="n">_collector</span><span class="o">;</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">prepare</span><span class="o">(</span><span class="n">Map</span> <span class="n">conf</span><span class="o">,</span> <span class="n">TopologyContext</span> <span class="n">context</span><span class="o">,</span> <span class="n">OutputCollectorBase</span> <span class="n">collector</span><span class="o">)</span> <span class="o">{</span>
<span class="n">_collector</span> <span class="o">=</span> <span class="n">collector</span><span class="o">;</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">execute</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">input</span><span class="o">)</span> <span class="o">{</span>
<span class="kt">int</span> <span class="n">val</span> <span class="o">=</span> <span class="n">input</span><span class="o">.</span><span class="na">getInteger</span><span class="o">(</span><span class="mi">0</span><span class="o">);</span>
<span class="n">_collector</span><span class="o">.</span><span class="na">emit</span><span class="o">(</span><span class="n">input</span><span class="o">,</span> <span class="k">new</span> <span class="n">Values</span><span class="o">(</span><span class="n">val</span><span class="o">*</span><span class="mi">2</span><span class="o">,</span> <span class="n">val</span><span class="o">*</span><span class="mi">3</span><span class="o">));</span>
<span class="n">_collector</span><span class="o">.</span><span class="na">ack</span><span class="o">(</span><span class="n">input</span><span class="o">);</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">declareOutputFields</span><span class="o">(</span><span class="n">OutputFieldsDeclarer</span> <span class="n">declarer</span><span class="o">)</span> <span class="o">{</span>
<span class="n">declarer</span><span class="o">.</span><span class="na">declare</span><span class="o">(</span><span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"double"</span><span class="o">,</span> <span class="s">"triple"</span><span class="o">));</span>
<span class="o">}</span>
<span class="o">}</span>
</code></pre></div>
<p>The <code>declareOutputFields</code> function declares the output fields <code>[&quot;double&quot;, &quot;triple&quot;]</code> for the component. The rest of the bolt will be explained in the upcoming sections.</p>
<h2 id="a-simple-topology">A simple topology</h2>
<p>Let&#39;s take a look at a simple topology to explore the concepts more and see how the code shapes up. Let&#39;s look at the <code>ExclamationTopology</code> definition from storm-starter:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">TopologyBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="k">new</span> <span class="n">TopologyBuilder</span><span class="o">();</span>
<span class="n">builder</span><span class="o">.</span><span class="na">setSpout</span><span class="o">(</span><span class="s">"words"</span><span class="o">,</span> <span class="k">new</span> <span class="n">TestWordSpout</span><span class="o">(),</span> <span class="mi">10</span><span class="o">);</span>
<span class="n">builder</span><span class="o">.</span><span class="na">setBolt</span><span class="o">(</span><span class="s">"exclaim1"</span><span class="o">,</span> <span class="k">new</span> <span class="n">ExclamationBolt</span><span class="o">(),</span> <span class="mi">3</span><span class="o">)</span>
<span class="o">.</span><span class="na">shuffleGrouping</span><span class="o">(</span><span class="s">"words"</span><span class="o">);</span>
<span class="n">builder</span><span class="o">.</span><span class="na">setBolt</span><span class="o">(</span><span class="s">"exclaim2"</span><span class="o">,</span> <span class="k">new</span> <span class="n">ExclamationBolt</span><span class="o">(),</span> <span class="mi">2</span><span class="o">)</span>
<span class="o">.</span><span class="na">shuffleGrouping</span><span class="o">(</span><span class="s">"exclaim1"</span><span class="o">);</span>
</code></pre></div>
<p>This topology contains a spout and two bolts. The spout emits words, and each bolt appends the string &quot;!!!&quot; to its input. The nodes are arranged in a line: the spout emits to the first bolt which then emits to the second bolt. If the spout emits the tuples [&quot;bob&quot;] and [&quot;john&quot;], then the second bolt will emit the words [&quot;bob!!!!!!&quot;] and [&quot;john!!!!!!&quot;].</p>
<p>This code defines the nodes using the <code>setSpout</code> and <code>setBolt</code> methods. These methods take as input a user-specified id, an object containing the processing logic, and the amount of parallelism you want for the node. In this example, the spout is given id &quot;words&quot; and the bolts are given ids &quot;exclaim1&quot; and &quot;exclaim2&quot;. </p>
<p>The object containing the processing logic implements the <a href="javadocs/org/apache/storm/topology/IRichSpout.html">IRichSpout</a> interface for spouts and the <a href="javadocs/org/apache/storm/topology/IRichBolt.html">IRichBolt</a> interface for bolts.</p>
<p>The last parameter, how much parallelism you want for the node, is optional. It indicates how many threads should execute that component across the cluster. If you omit it, Storm will only allocate one thread for that node.</p>
<p><code>setBolt</code> returns an <a href="javadocs/org/apache/storm/topology/InputDeclarer.html">InputDeclarer</a> object that is used to define the inputs to the Bolt. Here, component &quot;exclaim1&quot; declares that it wants to read all the tuples emitted by component &quot;words&quot; using a shuffle grouping, and component &quot;exclaim2&quot; declares that it wants to read all the tuples emitted by component &quot;exclaim1&quot; using a shuffle grouping. &quot;shuffle grouping&quot; means that tuples should be randomly distributed from the input tasks to the bolt&#39;s tasks. There are many ways to group data between components. These will be explained in a few sections.</p>
<p>If you wanted component &quot;exclaim2&quot; to read all the tuples emitted by both component &quot;words&quot; and component &quot;exclaim1&quot;, you would write component &quot;exclaim2&quot;&#39;s definition like this:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">builder</span><span class="o">.</span><span class="na">setBolt</span><span class="o">(</span><span class="s">"exclaim2"</span><span class="o">,</span> <span class="k">new</span> <span class="n">ExclamationBolt</span><span class="o">(),</span> <span class="mi">5</span><span class="o">)</span>
<span class="o">.</span><span class="na">shuffleGrouping</span><span class="o">(</span><span class="s">"words"</span><span class="o">)</span>
<span class="o">.</span><span class="na">shuffleGrouping</span><span class="o">(</span><span class="s">"exclaim1"</span><span class="o">);</span>
</code></pre></div>
<p>As you can see, input declarations can be chained to specify multiple sources for the Bolt.</p>
<p>Let&#39;s dig into the implementations of the spouts and bolts in this topology. Spouts are responsible for emitting new messages into the topology. <code>TestWordSpout</code> in this topology emits a random word from the list [&quot;nathan&quot;, &quot;mike&quot;, &quot;jackson&quot;, &quot;golda&quot;, &quot;bertels&quot;] as a 1-tuple every 100ms. The implementation of <code>nextTuple()</code> in TestWordSpout looks like this:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kt">void</span> <span class="nf">nextTuple</span><span class="o">()</span> <span class="o">{</span>
<span class="n">Utils</span><span class="o">.</span><span class="na">sleep</span><span class="o">(</span><span class="mi">100</span><span class="o">);</span>
<span class="kd">final</span> <span class="n">String</span><span class="o">[]</span> <span class="n">words</span> <span class="o">=</span> <span class="k">new</span> <span class="n">String</span><span class="o">[]</span> <span class="o">{</span><span class="s">"nathan"</span><span class="o">,</span> <span class="s">"mike"</span><span class="o">,</span> <span class="s">"jackson"</span><span class="o">,</span> <span class="s">"golda"</span><span class="o">,</span> <span class="s">"bertels"</span><span class="o">};</span>
<span class="kd">final</span> <span class="n">Random</span> <span class="n">rand</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Random</span><span class="o">();</span>
<span class="kd">final</span> <span class="n">String</span> <span class="n">word</span> <span class="o">=</span> <span class="n">words</span><span class="o">[</span><span class="n">rand</span><span class="o">.</span><span class="na">nextInt</span><span class="o">(</span><span class="n">words</span><span class="o">.</span><span class="na">length</span><span class="o">)];</span>
<span class="n">_collector</span><span class="o">.</span><span class="na">emit</span><span class="o">(</span><span class="k">new</span> <span class="n">Values</span><span class="o">(</span><span class="n">word</span><span class="o">));</span>
<span class="o">}</span>
</code></pre></div>
<p>As you can see, the implementation is very straightforward.</p>
<p><code>ExclamationBolt</code> appends the string &quot;!!!&quot; to its input. Let&#39;s take a look at the full implementation for <code>ExclamationBolt</code>:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">ExclamationBolt</span> <span class="kd">implements</span> <span class="n">IRichBolt</span> <span class="o">{</span>
<span class="n">OutputCollector</span> <span class="n">_collector</span><span class="o">;</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">prepare</span><span class="o">(</span><span class="n">Map</span> <span class="n">conf</span><span class="o">,</span> <span class="n">TopologyContext</span> <span class="n">context</span><span class="o">,</span> <span class="n">OutputCollector</span> <span class="n">collector</span><span class="o">)</span> <span class="o">{</span>
<span class="n">_collector</span> <span class="o">=</span> <span class="n">collector</span><span class="o">;</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">execute</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">)</span> <span class="o">{</span>
<span class="n">_collector</span><span class="o">.</span><span class="na">emit</span><span class="o">(</span><span class="n">tuple</span><span class="o">,</span> <span class="k">new</span> <span class="n">Values</span><span class="o">(</span><span class="n">tuple</span><span class="o">.</span><span class="na">getString</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="o">+</span> <span class="s">"!!!"</span><span class="o">));</span>
<span class="n">_collector</span><span class="o">.</span><span class="na">ack</span><span class="o">(</span><span class="n">tuple</span><span class="o">);</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">cleanup</span><span class="o">()</span> <span class="o">{</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">declareOutputFields</span><span class="o">(</span><span class="n">OutputFieldsDeclarer</span> <span class="n">declarer</span><span class="o">)</span> <span class="o">{</span>
<span class="n">declarer</span><span class="o">.</span><span class="na">declare</span><span class="o">(</span><span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"word"</span><span class="o">));</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="n">Map</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Object</span><span class="o">&gt;</span> <span class="nf">getComponentConfiguration</span><span class="o">()</span> <span class="o">{</span>
<span class="k">return</span> <span class="kc">null</span><span class="o">;</span>
<span class="o">}</span>
<span class="o">}</span>
</code></pre></div>
<p>The <code>prepare</code> method provides the bolt with an <code>OutputCollector</code> that is used for emitting tuples from this bolt. Tuples can be emitted at anytime from the bolt -- in the <code>prepare</code>, <code>execute</code>, or <code>cleanup</code> methods, or even asynchronously in another thread. This <code>prepare</code> implementation simply saves the <code>OutputCollector</code> as an instance variable to be used later on in the <code>execute</code> method.</p>
<p>The <code>execute</code> method receives a tuple from one of the bolt&#39;s inputs. The <code>ExclamationBolt</code> grabs the first field from the tuple and emits a new tuple with the string &quot;!!!&quot; appended to it. If you implement a bolt that subscribes to multiple input sources, you can find out which component the <a href="/javadoc/apidocs/org/apache/storm/tuple/Tuple.html">Tuple</a> came from by using the <code>Tuple#getSourceComponent</code> method.</p>
<p>There&#39;s a few other things going on in the <code>execute</code> method, namely that the input tuple is passed as the first argument to <code>emit</code> and the input tuple is acked on the final line. These are part of Storm&#39;s reliability API for guaranteeing no data loss and will be explained later in this tutorial. </p>
<p>The <code>cleanup</code> method is called when a Bolt is being shutdown and should cleanup any resources that were opened. There&#39;s no guarantee that this method will be called on the cluster: for example, if the machine the task is running on blows up, there&#39;s no way to invoke the method. The <code>cleanup</code> method is intended for when you run topologies in <a href="Local-mode.html">local mode</a> (where a Storm cluster is simulated in process), and you want to be able to run and kill many topologies without suffering any resource leaks.</p>
<p>The <code>declareOutputFields</code> method declares that the <code>ExclamationBolt</code> emits 1-tuples with one field called &quot;word&quot;.</p>
<p>The <code>getComponentConfiguration</code> method allows you to configure various aspects of how this component runs. This is a more advanced topic that is explained further on <a href="Configuration.html">Configuration</a>.</p>
<p>Methods like <code>cleanup</code> and <code>getComponentConfiguration</code> are often not needed in a bolt implementation. You can define bolts more succinctly by using a base class that provides default implementations where appropriate. <code>ExclamationBolt</code> can be written more succinctly by extending <code>BaseRichBolt</code>, like so:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">ExclamationBolt</span> <span class="kd">extends</span> <span class="n">BaseRichBolt</span> <span class="o">{</span>
<span class="n">OutputCollector</span> <span class="n">_collector</span><span class="o">;</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">prepare</span><span class="o">(</span><span class="n">Map</span> <span class="n">conf</span><span class="o">,</span> <span class="n">TopologyContext</span> <span class="n">context</span><span class="o">,</span> <span class="n">OutputCollector</span> <span class="n">collector</span><span class="o">)</span> <span class="o">{</span>
<span class="n">_collector</span> <span class="o">=</span> <span class="n">collector</span><span class="o">;</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">execute</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">)</span> <span class="o">{</span>
<span class="n">_collector</span><span class="o">.</span><span class="na">emit</span><span class="o">(</span><span class="n">tuple</span><span class="o">,</span> <span class="k">new</span> <span class="n">Values</span><span class="o">(</span><span class="n">tuple</span><span class="o">.</span><span class="na">getString</span><span class="o">(</span><span class="mi">0</span><span class="o">)</span> <span class="o">+</span> <span class="s">"!!!"</span><span class="o">));</span>
<span class="n">_collector</span><span class="o">.</span><span class="na">ack</span><span class="o">(</span><span class="n">tuple</span><span class="o">);</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">declareOutputFields</span><span class="o">(</span><span class="n">OutputFieldsDeclarer</span> <span class="n">declarer</span><span class="o">)</span> <span class="o">{</span>
<span class="n">declarer</span><span class="o">.</span><span class="na">declare</span><span class="o">(</span><span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"word"</span><span class="o">));</span>
<span class="o">}</span>
<span class="o">}</span>
</code></pre></div>
<h2 id="running-exclamationtopology-in-local-mode">Running ExclamationTopology in local mode</h2>
<p>Let&#39;s see how to run the <code>ExclamationTopology</code> in local mode and see that it&#39;s working.</p>
<p>Storm has two modes of operation: local mode and distributed mode. In local mode, Storm executes completely in process by simulating worker nodes with threads. Local mode is useful for testing and development of topologies. When you run the topologies in storm-starter, they&#39;ll run in local mode and you&#39;ll be able to see what messages each component is emitting. You can read more about running topologies in local mode on <a href="Local-mode.html">Local mode</a>.</p>
<p>In distributed mode, Storm operates as a cluster of machines. When you submit a topology to the master, you also submit all the code necessary to run the topology. The master will take care of distributing your code and allocating workers to run your topology. If workers go down, the master will reassign them somewhere else. You can read more about running topologies on a cluster on <a href="Running-topologies-on-a-production-cluster.html">Running topologies on a production cluster</a>]. </p>
<p>Here&#39;s the code that runs <code>ExclamationTopology</code> in local mode:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">Config</span> <span class="n">conf</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Config</span><span class="o">();</span>
<span class="n">conf</span><span class="o">.</span><span class="na">setDebug</span><span class="o">(</span><span class="kc">true</span><span class="o">);</span>
<span class="n">conf</span><span class="o">.</span><span class="na">setNumWorkers</span><span class="o">(</span><span class="mi">2</span><span class="o">);</span>
<span class="n">LocalCluster</span> <span class="n">cluster</span> <span class="o">=</span> <span class="k">new</span> <span class="n">LocalCluster</span><span class="o">();</span>
<span class="n">cluster</span><span class="o">.</span><span class="na">submitTopology</span><span class="o">(</span><span class="s">"test"</span><span class="o">,</span> <span class="n">conf</span><span class="o">,</span> <span class="n">builder</span><span class="o">.</span><span class="na">createTopology</span><span class="o">());</span>
<span class="n">Utils</span><span class="o">.</span><span class="na">sleep</span><span class="o">(</span><span class="mi">10000</span><span class="o">);</span>
<span class="n">cluster</span><span class="o">.</span><span class="na">killTopology</span><span class="o">(</span><span class="s">"test"</span><span class="o">);</span>
<span class="n">cluster</span><span class="o">.</span><span class="na">shutdown</span><span class="o">();</span>
</code></pre></div>
<p>First, the code defines an in-process cluster by creating a <code>LocalCluster</code> object. Submitting topologies to this virtual cluster is identical to submitting topologies to distributed clusters. It submits a topology to the <code>LocalCluster</code> by calling <code>submitTopology</code>, which takes as arguments a name for the running topology, a configuration for the topology, and then the topology itself.</p>
<p>The name is used to identify the topology so that you can kill it later on. A topology will run indefinitely until you kill it.</p>
<p>The configuration is used to tune various aspects of the running topology. The two configurations specified here are very common:</p>
<ol>
<li><strong>TOPOLOGY_WORKERS</strong> (set with <code>setNumWorkers</code>) specifies how many <em>processes</em> you want allocated around the cluster to execute the topology. Each component in the topology will execute as many <em>threads</em>. The number of threads allocated to a given component is configured through the <code>setBolt</code> and <code>setSpout</code> methods. Those <em>threads</em> exist within worker <em>processes</em>. Each worker <em>process</em> contains within it some number of <em>threads</em> for some number of components. For instance, you may have 300 threads specified across all your components and 50 worker processes specified in your config. Each worker process will execute 6 threads, each of which of could belong to a different component. You tune the performance of Storm topologies by tweaking the parallelism for each component and the number of worker processes those threads should run within.</li>
<li><strong>TOPOLOGY_DEBUG</strong> (set with <code>setDebug</code>), when set to true, tells Storm to log every message every emitted by a component. This is useful in local mode when testing topologies, but you probably want to keep this turned off when running topologies on the cluster.</li>
</ol>
<p>There&#39;s many other configurations you can set for the topology. The various configurations are detailed on <a href="javadocs/org/apache/storm/Config.html">the Javadoc for Config</a>.</p>
<p>To learn about how to set up your development environment so that you can run topologies in local mode (such as in Eclipse), see <a href="Creating-a-new-Storm-project.html">Creating a new Storm project</a>.</p>
<h2 id="stream-groupings">Stream groupings</h2>
<p>A stream grouping tells a topology how to send tuples between two components. Remember, spouts and bolts execute in parallel as many tasks across the cluster. If you look at how a topology is executing at the task level, it looks something like this:</p>
<p><img src="images/topology-tasks.png" alt="Tasks in a topology"></p>
<p>When a task for Bolt A emits a tuple to Bolt B, which task should it send the tuple to?</p>
<p>A &quot;stream grouping&quot; answers this question by telling Storm how to send tuples between sets of tasks. Before we dig into the different kinds of stream groupings, let&#39;s take a look at another topology from <a href="http://github.com/apache/storm/blob/1.2.3/examples/storm-starter">storm-starter</a>. This <a href="http://github.com/apache/storm/blob/v1.2.3/examples/storm-starter/src/jvm/org/apache/storm/starter/WordCountTopology.java">WordCountTopology</a> reads sentences off of a spout and streams out of <code>WordCountBolt</code> the total number of times it has seen that word before:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">TopologyBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="k">new</span> <span class="n">TopologyBuilder</span><span class="o">();</span>
<span class="n">builder</span><span class="o">.</span><span class="na">setSpout</span><span class="o">(</span><span class="s">"sentences"</span><span class="o">,</span> <span class="k">new</span> <span class="n">RandomSentenceSpout</span><span class="o">(),</span> <span class="mi">5</span><span class="o">);</span>
<span class="n">builder</span><span class="o">.</span><span class="na">setBolt</span><span class="o">(</span><span class="s">"split"</span><span class="o">,</span> <span class="k">new</span> <span class="n">SplitSentence</span><span class="o">(),</span> <span class="mi">8</span><span class="o">)</span>
<span class="o">.</span><span class="na">shuffleGrouping</span><span class="o">(</span><span class="s">"sentences"</span><span class="o">);</span>
<span class="n">builder</span><span class="o">.</span><span class="na">setBolt</span><span class="o">(</span><span class="s">"count"</span><span class="o">,</span> <span class="k">new</span> <span class="n">WordCount</span><span class="o">(),</span> <span class="mi">12</span><span class="o">)</span>
<span class="o">.</span><span class="na">fieldsGrouping</span><span class="o">(</span><span class="s">"split"</span><span class="o">,</span> <span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"word"</span><span class="o">));</span>
</code></pre></div>
<p><code>SplitSentence</code> emits a tuple for each word in each sentence it receives, and <code>WordCount</code> keeps a map in memory from word to count. Each time <code>WordCount</code> receives a word, it updates its state and emits the new word count.</p>
<p>There&#39;s a few different kinds of stream groupings.</p>
<p>The simplest kind of grouping is called a &quot;shuffle grouping&quot; which sends the tuple to a random task. A shuffle grouping is used in the <code>WordCountTopology</code> to send tuples from <code>RandomSentenceSpout</code> to the <code>SplitSentence</code> bolt. It has the effect of evenly distributing the work of processing the tuples across all of <code>SplitSentence</code> bolt&#39;s tasks.</p>
<p>A more interesting kind of grouping is the &quot;fields grouping&quot;. A fields grouping is used between the <code>SplitSentence</code> bolt and the <code>WordCount</code> bolt. It is critical for the functioning of the <code>WordCount</code> bolt that the same word always go to the same task. Otherwise, more than one task will see the same word, and they&#39;ll each emit incorrect values for the count since each has incomplete information. A fields grouping lets you group a stream by a subset of its fields. This causes equal values for that subset of fields to go to the same task. Since <code>WordCount</code> subscribes to <code>SplitSentence</code>&#39;s output stream using a fields grouping on the &quot;word&quot; field, the same word always goes to the same task and the bolt produces the correct output.</p>
<p>Fields groupings are the basis of implementing streaming joins and streaming aggregations as well as a plethora of other use cases. Underneath the hood, fields groupings are implemented using mod hashing.</p>
<p>There&#39;s a few other kinds of stream groupings. You can read more about them on <a href="Concepts.html">Concepts</a>. </p>
<h2 id="defining-bolts-in-other-languages">Defining Bolts in other languages</h2>
<p>Bolts can be defined in any language. Bolts written in another language are executed as subprocesses, and Storm communicates with those subprocesses with JSON messages over stdin/stdout. The communication protocol just requires an ~100 line adapter library, and Storm ships with adapter libraries for Ruby, Python, and Fancy. </p>
<p>Here&#39;s the definition of the <code>SplitSentence</code> bolt from <code>WordCountTopology</code>:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">SplitSentence</span> <span class="kd">extends</span> <span class="n">ShellBolt</span> <span class="kd">implements</span> <span class="n">IRichBolt</span> <span class="o">{</span>
<span class="kd">public</span> <span class="nf">SplitSentence</span><span class="o">()</span> <span class="o">{</span>
<span class="kd">super</span><span class="o">(</span><span class="s">"python"</span><span class="o">,</span> <span class="s">"splitsentence.py"</span><span class="o">);</span>
<span class="o">}</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">declareOutputFields</span><span class="o">(</span><span class="n">OutputFieldsDeclarer</span> <span class="n">declarer</span><span class="o">)</span> <span class="o">{</span>
<span class="n">declarer</span><span class="o">.</span><span class="na">declare</span><span class="o">(</span><span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"word"</span><span class="o">));</span>
<span class="o">}</span>
<span class="o">}</span>
</code></pre></div>
<p><code>SplitSentence</code> overrides <code>ShellBolt</code> and declares it as running using <code>python</code> with the arguments <code>splitsentence.py</code>. Here&#39;s the implementation of <code>splitsentence.py</code>:</p>
<div class="highlight"><pre><code class="language-python" data-lang="python"><span class="kn">import</span> <span class="nn">storm</span>
<span class="k">class</span> <span class="nc">SplitSentenceBolt</span><span class="p">(</span><span class="n">storm</span><span class="o">.</span><span class="n">BasicBolt</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tup</span><span class="p">):</span>
<span class="n">words</span> <span class="o">=</span> <span class="n">tup</span><span class="o">.</span><span class="n">values</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s">" "</span><span class="p">)</span>
<span class="k">for</span> <span class="n">word</span> <span class="ow">in</span> <span class="n">words</span><span class="p">:</span>
<span class="n">storm</span><span class="o">.</span><span class="n">emit</span><span class="p">([</span><span class="n">word</span><span class="p">])</span>
<span class="n">SplitSentenceBolt</span><span class="p">()</span><span class="o">.</span><span class="n">run</span><span class="p">()</span>
</code></pre></div>
<p>For more information on writing spouts and bolts in other languages, and to learn about how to create topologies in other languages (and avoid the JVM completely), see <a href="Using-non-JVM-languages-with-Storm.html">Using non-JVM languages with Storm</a>.</p>
<h2 id="guaranteeing-message-processing">Guaranteeing message processing</h2>
<p>Earlier on in this tutorial, we skipped over a few aspects of how tuples are emitted. Those aspects were part of Storm&#39;s reliability API: how Storm guarantees that every message coming off a spout will be fully processed. See <a href="Guaranteeing-message-processing.html">Guaranteeing message processing</a> for information on how this works and what you have to do as a user to take advantage of Storm&#39;s reliability capabilities.</p>
<h2 id="transactional-topologies">Transactional topologies</h2>
<p>Storm guarantees that every message will be played through the topology at least once. A common question asked is &quot;how do you do things like counting on top of Storm? Won&#39;t you overcount?&quot; Storm has a feature called transactional topologies that let you achieve exactly-once messaging semantics for most computations. Read more about transactional topologies <a href="Transactional-topologies.html">here</a>. </p>
<h2 id="distributed-rpc">Distributed RPC</h2>
<p>This tutorial showed how to do basic stream processing on top of Storm. There&#39;s lots more things you can do with Storm&#39;s primitives. One of the most interesting applications of Storm is Distributed RPC, where you parallelize the computation of intense functions on the fly. Read more about Distributed RPC <a href="Distributed-RPC.html">here</a>. </p>
<h2 id="conclusion">Conclusion</h2>
<p>This tutorial gave a broad overview of developing, testing, and deploying Storm topologies. The rest of the documentation dives deeper into all the aspects of using Storm.</p>
</div>
</div>
</div>
</div>
<footer>
<div class="container-fluid">
<div class="row">
<div class="col-md-3">
<div class="footer-widget">
<h5>Meetups</h5>
<ul class="latest-news">
<li><a href="http://www.meetup.com/Apache-Storm-Apache-Kafka/">Apache Storm & Apache Kafka</a> <span class="small">(Sunnyvale, CA)</span></li>
<li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Apache Storm & Kafka Users</a> <span class="small">(Seattle, WA)</span></li>
<li><a href="http://www.meetup.com/New-York-City-Storm-User-Group/">NYC Storm User Group</a> <span class="small">(New York, NY)</span></li>
<li><a href="http://www.meetup.com/Bay-Area-Stream-Processing">Bay Area Stream Processing</a> <span class="small">(Emeryville, CA)</span></li>
<li><a href="http://www.meetup.com/Boston-Storm-Users/">Boston Realtime Data</a> <span class="small">(Boston, MA)</span></li>
<li><a href="http://www.meetup.com/storm-london">London Storm User Group</a> <span class="small">(London, UK)</span></li>
<!-- <li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Seatle, WA</a> <span class="small">(27 Jun 2015)</span></li> -->
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>About Apache Storm</h5>
<p>Apache Storm integrates with any queueing system and any database system. Apache Storm's spout abstraction makes it easy to integrate a new queuing system. Likewise, integrating Apache Storm with database systems is easy.</p>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>First Look</h5>
<ul class="footer-list">
<li><a href="/releases/current/Rationale.html">Rationale</a></li>
<li><a href="/releases/current/Tutorial.html">Tutorial</a></li>
<li><a href="/releases/current/Setting-up-development-environment.html">Setting up development environment</a></li>
<li><a href="/releases/current/Creating-a-new-Storm-project.html">Creating a new Apache Storm project</a></li>
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>Documentation</h5>
<ul class="footer-list">
<li><a href="/releases/current/index.html">Index</a></li>
<li><a href="/releases/current/javadocs/index.html">Javadoc</a></li>
<li><a href="/releases/current/FAQ.html">FAQ</a></li>
</ul>
</div>
</div>
</div>
<hr/>
<div class="row">
<div class="col-md-12">
<p align="center">Copyright © 2019 <a href="http://www.apache.org">Apache Software Foundation</a>. All Rights Reserved.
<br>Apache Storm, Apache, the Apache feather logo, and the Apache Storm project logos are trademarks of The Apache Software Foundation.
<br>All other marks mentioned may be trademarks or registered trademarks of their respective owners.</p>
</div>
</div>
</div>
</footer>
<!--Footer End-->
<!-- Scroll to top -->
<span class="totop"><a href="#"><i class="fa fa-angle-up"></i></a></span>
</body>
</html>