Jekyll2024-03-06T07:32:41+01:00https://www.michael-noll.com/feed.xmlMichael G. NollI am a software engineer turned product manager. Currently focusing on product & technology strategy and competitive analysis at Confluent (USA), the company founded by the creators of the large-scale data processing software Apache Kafka.The Future of SQL: Databases Meet Stream Processing2021-11-03T10:00:00+01:002021-11-03T10:00:00+01:00https://www.michael-noll.com/blog/2021/11/03/the-future-of-sql-databases-meet-stream-processing<p>The database language <a href="https://en.wikipedia.org/wiki/SQL">SQL</a> has proven to be an invaluable asset for most developers building software applications. Yet, the world as we know it has changed dramatically since SQL was created in 1974. Together with my colleague <a href="https://twitter.com/matthiasjsax">Dr. Matthias J. Sax</a> (ASF committer and PMC for Apache Kafka, Apache Flink, and Apache Storm) I wrote an article on how the future of SQL is being shaped by changes in how data—and notably data in motion—is used to build applications today.</p>
<!-- more -->
<p>Head over to <a href="https://www.confluent.io/blog/databases-meet-stream-processing-the-future-of-sql/"><strong>The Future of SQL: Databases Meet Stream Processing</strong></a> on the Confluent blog to start reading.</p>
<p><img src="/assets/uploads/traditional-vs-streaming-database.jpg" alt="Traditional Databases vs. Streaming Databases" />
<em>Figure: Today’s world never sleeps. As software engineers, we are increasingly building architectures in which automated systems drive other automated systems directly, 24x7. And this requires a change of engineering strategy: The modern database, as shown on the right, must support a “data-active, query-passive” approach in which queries execute and passively wait for new data to arrive from upstream queries and data sources.</em></p>
<p>We start with a brief history of SQL and how it’s been used with data systems through to the modern day and then cover the rise of Apache Kafka and data in motion. We then take a deep dive into several key areas where SQL—originally designed for data at rest—needs to meet data in motion so that we engineers can build the modern architectures that companies need today and in the future.</p>
<p><img src="/assets/uploads/joining-stream-to-table.jpg" alt="Joining a Data Streae to Data Table" />
<em>Figure: An example of where data in motion meets data at rest: a stream of customer location changes is continuously being joined to a table with customer profile data. See the linked article for full explanation.</em></p>
<p>If you have finished reading the article, you might wonder what is happening next?
There’s a lot of activity in the technical community in this area. Matthias and I have been participating on behalf of Confluent in <a href="https://www.incits.org/">INCITS</a>, the U.S. national body for a variety of technical standards, including the SQL standard. Specifically, we have been working in the technical committee <a href="https://www.incits.org/committees/dm32">DM32 (Data Management and Interchange)</a> that develops standards for database languages. Here, we are collaborating in a Streaming SQL working group together with vendors of traditional databases such as Microsoft, Oracle, and IBM, cloud database players like Snowflake., and vendors of stream processing systems like Alibaba and Google. This list of participants, many of which are industry leaders with both technical pedigree as well as strong market reputation, presents a clear validation of the increasing importance of event streaming and streaming databases in modern software architectures.</p>
<p>So if you want to learn even more, my article is actually part one of a series of Readings in Streaming Database Systems. Feel free to check out the full series:</p>
<ol>
<li><a href="https://www.confluent.io/blog/streaming-databases-vs-traditional-databases/">Overview: Readings in Streaming Database Systems</a></li>
<li><a href="https://www.confluent.io/blog/databases-meet-stream-processing-the-future-of-sql/">Part 1: The Future of SQL: Databases Meet Stream Processing</a> (my article)</li>
<li><a href="https://www.confluent.io/blog/streaming-database-design-principles-and-guarantees/">Part 2: 4 Key Design Principles and Guarantees of Streaming Databases</a></li>
<li><a href="https://www.confluent.io/blog/streaming-databases-cloud-data-in-motion-never-rests/">Part 3: How Do You Change a Never-Ending Query?</a></li>
</ol>
<p>Enjoy!</p>The database language SQL has proven to be an invaluable asset for most developers building software applications. Yet, the world as we know it has changed dramatically since SQL was created in 1974. Together with my colleague Dr. Matthias J. Sax (ASF committer and PMC for Apache Kafka, Apache Flink, and Apache Storm) I wrote an article on how the future of SQL is being shaped by changes in how data—and notably data in motion—is used to build applications today.What Every Software Engineer Should Know about Apache Kafka: Events, Streams, Tables, Storage, Processing, And More2020-01-16T10:00:00+01:002020-01-16T10:00:00+01:00https://www.michael-noll.com/blog/2020/01/16/what-every-software-engineer-should-know-about-apache-kafka-fundamentals<p>To help fellow engineers wrap their head around Apache Kafka and event streaming, I wrote a 4-part series on the <a href="https://www.confluent.io">Confluent</a> blog on Kafka’s core fundamentals. In the series, we explore Kafka’s storage and processing layers and how they interrelate, featuring Kafka Streams and ksqlDB.</p>
<!-- more -->
<p><img src="/assets/uploads/streams-tables-chess-animation-large.gif" alt="Streams vs. Tables in Chess" /></p>
<p>In the first part, I begin with an overview of events, streams, tables, and the stream-table duality to set the stage. The subsequent parts take a closer look at Kafka’s storage layer, which is the distributed “filesystem” for streams and tables, where we learn about topics and partitions. Then, I move to the processing layer on top and dive into parallel processing of streams and tables, elastic scalability, fault tolerance, and much more. The series is related to my original article, <a href="/blog/2018/04/05/of-stream-and-tables-in-kafka-and-stream-processing-part1/">Of Streams and Tables in Kafka and Stream Processing, Part 1</a>, but is both broader and deeper.</p>
<ol>
<li><a href="https://www.confluent.io/blog/kafka-streams-tables-part-1-event-streaming/">Streams and Tables in Apache Kafka: A Primer</a></li>
<li><a href="https://confluent.io/blog/kafka-streams-tables-part-2-topics-partitions-and-storage-fundamentals/">Streams and Tables in Apache Kafka: Topics, Partitions, and Storage Fundamentals</a></li>
<li><a href="https://confluent.io/blog/kafka-streams-tables-part-3-event-processing-fundamentals/">Streams and Tables in Apache Kafka: Processing Fundamentals with Kafka Streams and ksqlDB</a></li>
<li><a href="https://www.confluent.io/blog/kafka-streams-tables-part-4-elasticity-fault-tolerance-advanced-concepts/">Streams and Tables in Apache Kafka: Elasticity, Fault Tolerance, and other Advanced Concepts</a></li>
</ol>
<p>I hope you enjoy the walkthrough!</p>To help fellow engineers wrap their head around Apache Kafka and event streaming, I wrote a 4-part series on the Confluent blog on Kafka’s core fundamentals. In the series, we explore Kafka’s storage and processing layers and how they interrelate, featuring Kafka Streams and ksqlDB.Of Streams and Tables in Kafka and Stream Processing, Part 12018-04-05T17:55:00+02:002018-04-05T17:55:00+02:00https://www.michael-noll.com/blog/2018/04/05/of-stream-and-tables-in-kafka-and-stream-processing-part1<p>In this article, perhaps the first in a mini-series, I want to explain the concepts of streams and tables in stream processing and, specifically, in Apache Kafka. Hopefully, you will walk away with both a better theoretical understanding but also more tangible insights and ideas that will help you solve your current or next practical use case better, faster, or both.</p>
<!-- more -->
<div class="note">
<p>
<strong>Update (January 2020):</strong> I have since written a 4-part series on the <a href="https://www.confluent.io">Confluent</a> blog on Apache Kafka fundamentals, which goes beyond what I cover in this original article.
</p>
<p>
In the first part, I begin with an overview of events, streams, tables, and the stream-table duality to set the stage. The subsequent parts take a closer look at Kafka’s storage layer, which is the distributed "filesystem" for streams and tables, where we learn about topics and partitions. Then, I move to the processing layer on top and dive into parallel processing of streams and tables, elastic scalability, fault tolerance, and much more. The series is related to my original article below, but is both broader and deeper.
</p>
<p>
<ol>
<li><a href="https://www.confluent.io/blog/kafka-streams-tables-part-1-event-streaming/">Streams and Tables in Apache Kafka: A Primer</a></li>
<li><a href="https://confluent.io/blog/kafka-streams-tables-part-2-topics-partitions-and-storage-fundamentals/">Streams and Tables in Apache Kafka: Topics, Partitions, and Storage Fundamentals</a></li>
<li><a href="https://confluent.io/blog/kafka-streams-tables-part-3-event-processing-fundamentals/">Streams and Tables in Apache Kafka: Processing Fundamentals with Kafka Streams and ksqlDB</a></li>
<li><a href="https://www.confluent.io/blog/kafka-streams-tables-part-4-elasticity-fault-tolerance-advanced-concepts/">Streams and Tables in Apache Kafka: Elasticity, Fault Tolerance, and other Advanced Concepts</a></li>
</ol>
</p>
</div>
<ul id="markdown-toc">
<li><a href="#motivation-or-why-should-i-care" id="markdown-toc-motivation-or-why-should-i-care">Motivation, or: Why Should I Care?</a></li>
<li><a href="#streams-and-tables-in-plain-english" id="markdown-toc-streams-and-tables-in-plain-english">Streams and Tables in Plain English</a></li>
<li><a href="#illustrated-examples" id="markdown-toc-illustrated-examples">Illustrated Examples</a></li>
<li><a href="#streams-and-tables-in-kafka-english" id="markdown-toc-streams-and-tables-in-kafka-english">Streams and Tables in Kafka English</a></li>
<li><a href="#streams-and-tables-in-your-daily-development-work" id="markdown-toc-streams-and-tables-in-your-daily-development-work">Streams and Tables in Your Daily Development Work</a></li>
<li><a href="#a-closer-look-with-kafka-streams-ksql-and-analogies-in-scala" id="markdown-toc-a-closer-look-with-kafka-streams-ksql-and-analogies-in-scala">A Closer Look with Kafka Streams, KSQL, and Analogies in Scala</a> <ul>
<li><a href="#topics" id="markdown-toc-topics">Topics</a></li>
<li><a href="#streams" id="markdown-toc-streams">Streams</a></li>
<li><a href="#tables" id="markdown-toc-tables">Tables</a></li>
</ul>
</li>
<li><a href="#tables-stand-on-the-shoulders-of-stream-giants" id="markdown-toc-tables-stand-on-the-shoulders-of-stream-giants">Tables Stand on the Shoulders of Stream Giants</a> <ul>
<li><a href="#aggregations-in-kafka" id="markdown-toc-aggregations-in-kafka">Aggregations in Kafka</a></li>
<li><a href="#tables-are-aggregated-streams-input-stream--table" id="markdown-toc-tables-are-aggregated-streams-input-stream--table">Tables are Aggregated Streams (input stream → table)</a></li>
<li><a href="#tables-have-changelog-streams-table--output-stream" id="markdown-toc-tables-have-changelog-streams-table--output-stream">Tables have Changelog Streams (table → output stream)</a></li>
<li><a href="#the-stream-table-duality" id="markdown-toc-the-stream-table-duality">The Stream-Table Duality</a></li>
</ul>
</li>
<li><a href="#turning-the-database-inside-out" id="markdown-toc-turning-the-database-inside-out">Turning the Database Inside-Out</a></li>
<li><a href="#wrapping-up" id="markdown-toc-wrapping-up">Wrapping Up</a></li>
</ul>
<h1 id="motivation-or-why-should-i-care">Motivation, or: Why Should I Care?</h1>
<p>In my daily work I interact with many users regarding Apache Kafka and doing stream processing with Kafka through <a href="https://kafka.apache.org/documentation/streams/">Kafka’s Streams API</a> (aka Kafka Streams) and <a href="https://github.com/confluentinc/ksql">KSQL</a> (the streaming SQL engine for Kafka). Some users have a stream processing or Kafka background, some have their roots in RDBMS like Oracle and MySQL, some have neither.</p>
<p>One common question is, “What’s the difference between streams and tables?” In this article I want to give both a short TL;DR answer but also a longer answer so that you can get a deeper understanding. Some of the explanations below will be slightly simplified because that makes them easier to understand and also easier to remember (like how Newton’s simpler but less accurate gravity model is perfectly sufficient for most daily situations, saving you from having to jump straight to Einstein’s model of relativity; well, fortunately, stream processing is never <em>that</em> complicated anyways).</p>
<p>Another common question is, “Alright, but why should I care? How does this help me in my daily work?” In short, a lot! Once you start implementing your own first use cases, you will soon realize that, in practice, most streaming use cases actually require both streams <em>and</em> tables. Tables, as I will explain later, represent <em>state</em>. Whenever you are doing any <em>stateful</em> processing like joins (e.g., for real-time data enrichment by joining a stream of facts with dimension tables) and aggregations (e.g., computing 5-minute averages for key business metrics in real-time), then tables enter the streaming picture. And if they don’t, then this means you are in for a lot of DIY pain. Even the infamous WordCount example, probably the first Hello World you have encountered in this space, falls into the stateful category: it is an example of stateful processing where we aggregate a stream of text lines into a continuously updated table/map of word counts. So whether you are implementing a simple streaming WordCount or something more sophisticated like fraud detection, you want an easy-to-use stream processing solution with all batteries and core data structures (hint: streams and tables) included. You certainly don’t want to build needlessly complex architectures where you must stitch together a stream(-only) processing technology with a remote datastore like Cassandra or MySQL, and probably also having to add Hadoop/HDFS to enable fault-tolerant processing (3 things are 2 too many).</p>
<h1 id="streams-and-tables-in-plain-english">Streams and Tables in Plain English</h1>
<p>Here’s the best analogy I can think of:</p>
<ul>
<li>A <strong>stream</strong> in Kafka records the full history of world (or business) events <em>from the beginning of time to today</em>. It represents the past and the present. As we go from today to tomorrow, new events are constantly being added to the world’s history. This history is a sequence or “chain” of events, so you know which event happened before another event.</li>
<li>A <strong>table</strong> in Kafka is the state of the world <em>today</em>. It represents the present. It is an aggregation of the history of world events, and this aggregation is changing constantly as we go from today to tomorrow.</li>
</ul>
<p>Let’s use Chess as an example:</p>
<p><img src="/assets/uploads/streams-tables-chess-animation-large.gif" alt="Streams vs. Tables in Chess" /></p>
<p>And as an appetizer for a future blog post: If you have access to the full history of world events (stream), then you can of course generally reconstruct the state of the world <em>at any arbitrary time</em>, i.e. the table at an arbitrary time t in the stream, where t is not restrained to be only t=now. In other words, we can create “snapshots” of the world’s state (table) for any time t, such as 2560 BC when the Great Pyramid of Giza was built, or 1993 AC when the European Union was formed.</p>
<h1 id="illustrated-examples">Illustrated Examples</h1>
<p>The first use case example shows a <strong>stream</strong> of user geo-location updates that is being aggregated into a <strong>table</strong> that tracks the <em>current aka latest location for each user</em>. As I will explain later, this also happens to be the default table semantics when you are reading a Kafka topic directly into a table.</p>
<p><img src="/assets/uploads/stream-table-animation-latestLocation.gif" alt="First use case example (table tracks latest values per key)" /></p>
<p>The second use case example shows the same <strong>stream</strong> of user geo-location updates, but now the stream is being aggregated into a <strong>table</strong> that tracks the <em>number of visited locations for each user</em>. Because the aggregation function is different (here: counting) the contents of the table are different. More precisely, the values per key are different.</p>
<p><img src="/assets/uploads/stream-table-animation-numVisitedLocations.gif" alt="Second use case example (table tracks counts per key)" /></p>
<h1 id="streams-and-tables-in-kafka-english">Streams and Tables in Kafka English</h1>
<p>Before we dive into details let us start with a simplified summary.</p>
<p>A <strong>topic</strong> in Kafka is an unbounded sequence of key-value pairs. Keys and values are raw byte arrays, i.e. <code class="language-plaintext highlighter-rouge"><byte[], byte[]></code>.</p>
<p>A <strong>stream</strong> is a topic with a schema. Keys and values are no longer byte arrays but have specific types.</p>
<ul>
<li>Example: A <code class="language-plaintext highlighter-rouge"><byte[], byte[]></code> topic is read as a <code class="language-plaintext highlighter-rouge"><User, GeoLocation></code> stream of geo-location updates.</li>
</ul>
<p>A <strong>table</strong> is a, well, table in the ordinary sense of the word (I hear the happy fist pumps of those of you who are familiar with RDBMS but new to Kafka). But, seen through the lens of stream processing, a table is also an <em>aggregated stream</em> (you didn’t really expect we would just stop at “a table is a table”, did you?).</p>
<ul>
<li>
<p>Example: A <code class="language-plaintext highlighter-rouge"><User, GeoLocation></code> stream of geo-location updates is aggregated into a <code class="language-plaintext highlighter-rouge"><User, GeoLocation></code> table that tracks the latest location of each user. The aggregation step continuously UPSERTs values per key from the input stream to the table. We have seen this in the first illustrated example above.</p>
</li>
<li>
<p>Example: A <code class="language-plaintext highlighter-rouge"><User, GeoLocation></code> stream is aggregated into a <code class="language-plaintext highlighter-rouge"><User, Long></code> table that tracks the number of visited locations per user. The aggregation step continuously counts (and updates) the number of observed values per key in the table. We have seen this in the second illustrated example above.</p>
</li>
</ul>
<p>Think:</p>
<p><img src="/assets/uploads/topic-stream-table.png" alt="Topic vs. Stream vs. Table in Kafka)" /></p>
<p>Topics, streams, and tables have the following properties in Kafka:</p>
<table>
<thead>
<tr>
<th>Concept</th>
<th>Partitioned</th>
<th>Unbounded</th>
<th>Ordering</th>
<th>Mutable</th>
<th>Unique key constraint</th>
<th>Schema</th>
</tr>
</thead>
<tbody>
<tr>
<td>Topic</td>
<td>Yes</td>
<td>Yes</td>
<td>Yes</td>
<td>No</td>
<td>No</td>
<td>No (raw bytes)</td>
</tr>
<tr>
<td>Stream</td>
<td>Yes</td>
<td>Yes</td>
<td>Yes</td>
<td>No</td>
<td>No</td>
<td>Yes</td>
</tr>
<tr>
<td>Table</td>
<td>Yes</td>
<td>Yes</td>
<td>No</td>
<td>Yes</td>
<td>Yes</td>
<td>Yes</td>
</tr>
</tbody>
</table>
<p>Let’s see how topics, streams, and tables relate to Kafka’s Streams API and KSQL, and also draw analogies to programming languages (the analogies ignore, for example, that topics/streams/tables are partitioned):</p>
<table>
<thead>
<tr>
<th>Concept</th>
<th>Kafka Streams</th>
<th>KSQL</th>
<th>Java</th>
<th>Scala</th>
<th>Python</th>
</tr>
</thead>
<tbody>
<tr>
<td>Topic</td>
<td>-</td>
<td>-</td>
<td><code class="language-plaintext highlighter-rouge">List/Stream</code></td>
<td><code class="language-plaintext highlighter-rouge">List/Stream[(Array[Byte], Array[Byte])]</code></td>
<td><code class="language-plaintext highlighter-rouge">[]</code></td>
</tr>
<tr>
<td>Stream</td>
<td><code class="language-plaintext highlighter-rouge">KStream</code></td>
<td><code class="language-plaintext highlighter-rouge">STREAM</code></td>
<td><code class="language-plaintext highlighter-rouge">List/Stream</code></td>
<td><code class="language-plaintext highlighter-rouge">List/Stream[(K, V)]</code></td>
<td><code class="language-plaintext highlighter-rouge">[]</code></td>
</tr>
<tr>
<td>Table</td>
<td><code class="language-plaintext highlighter-rouge">KTable</code></td>
<td><code class="language-plaintext highlighter-rouge">TABLE</code></td>
<td><code class="language-plaintext highlighter-rouge">HashMap</code></td>
<td><code class="language-plaintext highlighter-rouge">mutable.Map[K, V]</code></td>
<td><code class="language-plaintext highlighter-rouge">{}</code></td>
</tr>
</tbody>
</table>
<h1 id="streams-and-tables-in-your-daily-development-work">Streams and Tables in Your Daily Development Work</h1>
<p>Let me make a final analogy. If you are a developer, you are very likely familiar with git. In git, the commit history represents the history of the repository (aka the world), and a checkout of the repository represents the state of the repository at a particular point in time. When you do a <code class="language-plaintext highlighter-rouge">git checkout <commit></code>, then git will dynamically compute the corresponding state aka checkout from the commit history; i.e., the checkout is an aggregation of the commit history. This is very similar to how Kafka computes tables dynamically from streams through aggregation.</p>
<table>
<thead>
<tr>
<th>Concept</th>
<th>Git</th>
</tr>
</thead>
<tbody>
<tr>
<td>Stream</td>
<td>commit history (<code class="language-plaintext highlighter-rouge">git log</code>)</td>
</tr>
<tr>
<td>Table</td>
<td>repo at commit (<code class="language-plaintext highlighter-rouge">git checkout <commit></code>)</td>
</tr>
</tbody>
</table>
<p>So much for a first overview. Now we can take a closer look.</p>
<h1 id="a-closer-look-with-kafka-streams-ksql-and-analogies-in-scala">A Closer Look with Kafka Streams, KSQL, and Analogies in Scala</h1>
<p>I’ll start each of the following sections with a Scala analogy (think: stream processing on a single machine) and the Scala REPL so that you can copy-paste and play around yourself, then I’ll explain how to do the same in Kafka Streams and KSQL (elastic, scalable, fault-tolerant stream processing on distributed machines). As I mentioned in the beginning, I slightly simplify the explanations below. For example, I will not cover the impact of partitioning in Kafka.</p>
<div class="note">
<strong>If you don't know Scala:</strong> Don't be dismayed! You don't need to understand the Scala analogies below in every detail. Just pay attention to which operations (like <code>map()</code>) are being chained together, what these operations represent (e.g. <code>reduceLeft()</code> represents an aggregation), and how the stream "chain" compares to the table "chain".
</div>
<h2 id="topics">Topics</h2>
<p>A topic in Kafka consists of key-value messages. The topic is agnostic to the serialization format or “type” of its messages: it treats message keys and message values universally as byte arrays aka <code class="language-plaintext highlighter-rouge">byte[]</code>. In other words, at this point we have no idea yet what’s in the data.</p>
<p>Kafka Streams and KSQL don’t have a concept of “a topic”. They only know about streams and tables. So I only show the Scala analogy for a topic here.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Scala analogy</span>
<span class="n">scala</span><span class="o">></span> <span class="k">val</span> <span class="nv">topic</span><span class="k">:</span> <span class="kt">Seq</span><span class="o">[(</span><span class="kt">Array</span><span class="o">[</span><span class="kt">Byte</span><span class="o">]</span>, <span class="kt">Array</span><span class="o">[</span><span class="kt">Byte</span><span class="o">])]</span> <span class="k">=</span> <span class="nc">Seq</span><span class="o">((</span><span class="nc">Array</span><span class="o">(</span><span class="mi">97</span><span class="o">,</span> <span class="mi">108</span><span class="o">,</span> <span class="mi">105</span><span class="o">,</span> <span class="mi">99</span><span class="o">,</span> <span class="mi">101</span><span class="o">),</span><span class="nc">Array</span><span class="o">(</span><span class="mi">80</span><span class="o">,</span> <span class="mi">97</span><span class="o">,</span> <span class="mi">114</span><span class="o">,</span> <span class="mi">105</span><span class="o">,</span> <span class="mi">115</span><span class="o">)),</span> <span class="o">(</span><span class="nc">Array</span><span class="o">(</span><span class="mi">98</span><span class="o">,</span> <span class="mi">111</span><span class="o">,</span> <span class="mi">98</span><span class="o">),</span><span class="nc">Array</span><span class="o">(</span><span class="mi">83</span><span class="o">,</span> <span class="mi">121</span><span class="o">,</span> <span class="mi">100</span><span class="o">,</span> <span class="mi">110</span><span class="o">,</span> <span class="mi">101</span><span class="o">,</span> <span class="mi">121</span><span class="o">)),</span> <span class="o">(</span><span class="nc">Array</span><span class="o">(</span><span class="mi">97</span><span class="o">,</span> <span class="mi">108</span><span class="o">,</span> <span class="mi">105</span><span class="o">,</span> <span class="mi">99</span><span class="o">,</span> <span class="mi">101</span><span class="o">),</span><span class="nc">Array</span><span class="o">(</span><span class="mi">82</span><span class="o">,</span> <span class="mi">111</span><span class="o">,</span> <span class="mi">109</span><span class="o">,</span> <span class="mi">101</span><span class="o">)),</span> <span class="o">(</span><span class="nc">Array</span><span class="o">(</span><span class="mi">98</span><span class="o">,</span> <span class="mi">111</span><span class="o">,</span> <span class="mi">98</span><span class="o">),</span><span class="nc">Array</span><span class="o">(</span><span class="mi">76</span><span class="o">,</span> <span class="mi">105</span><span class="o">,</span> <span class="mi">109</span><span class="o">,</span> <span class="mi">97</span><span class="o">)),</span> <span class="o">(</span><span class="nc">Array</span><span class="o">(</span><span class="mi">97</span><span class="o">,</span> <span class="mi">108</span><span class="o">,</span> <span class="mi">105</span><span class="o">,</span> <span class="mi">99</span><span class="o">,</span> <span class="mi">101</span><span class="o">),</span><span class="nc">Array</span><span class="o">(</span><span class="mi">66</span><span class="o">,</span> <span class="mi">101</span><span class="o">,</span> <span class="mi">114</span><span class="o">,</span> <span class="mi">108</span><span class="o">,</span> <span class="mi">105</span><span class="o">,</span> <span class="mi">110</span><span class="o">)))</span>
</code></pre></div></div>
<h2 id="streams">Streams</h2>
<p>We now read the topic into a stream by adding schema information (schema-on-read). In other words, we are turning the raw, untyped topic into a “typed topic” aka stream.</p>
<div class="note">
<strong>Schema-on-read vs. schema-on-write:</strong> Kafka and its topics are agnostic to the serialization format of your data. That's why you must specify the schema when you want to read the data into a stream or table. This is called schema-on-read. There are upsides and downsides to schema-on-read. Fortunately, you can find a middle ground between schema-on-read and schema-on-write by defining a contract for your data -- similarly how you probably have been defining API contracts all the time in your applications and services. This can be achieved by picking a structured yet evolvable data format such as <a href="https://avro.apache.org/">Apache Avro</a> and deploying a registry for your Avro schemas such as <a href="https://github.com/confluentinc/schema-registry">Confluent Schema Registry</a>. And yes, both Kafka Streams and KSQL support Avro, in case you are wondering.
</div>
<p>In Scala this is achieved by the <code class="language-plaintext highlighter-rouge">map()</code> operation below. In this example, we end up with a stream of <code class="language-plaintext highlighter-rouge"><String, String></code> pairs. Notice how we can now see what’s in the data.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Scala analogy</span>
<span class="n">scala</span><span class="o">></span> <span class="k">val</span> <span class="nv">stream</span> <span class="k">=</span> <span class="n">topic</span>
<span class="o">|</span> <span class="o">.</span><span class="py">map</span> <span class="o">{</span> <span class="nf">case</span> <span class="o">(</span><span class="n">k</span><span class="k">:</span> <span class="kt">Array</span><span class="o">[</span><span class="kt">Byte</span><span class="o">],</span> <span class="n">v</span><span class="k">:</span> <span class="kt">Array</span><span class="o">[</span><span class="kt">Byte</span><span class="o">])</span> <span class="k">=></span> <span class="k">new</span> <span class="nc">String</span><span class="o">(</span><span class="n">k</span><span class="o">)</span> <span class="o">-></span> <span class="k">new</span> <span class="nc">String</span><span class="o">(</span><span class="n">v</span><span class="o">)</span> <span class="o">}</span>
<span class="c1">// => stream: Seq[(String, String)] =</span>
<span class="c1">// List((alice,Paris), (bob,Sydney), (alice,Rome), (bob,Lima), (alice,Berlin))</span>
</code></pre></div></div>
<p>In Kafka Streams you read a topic into a <code class="language-plaintext highlighter-rouge">KStream</code> via <code class="language-plaintext highlighter-rouge">StreamsBuilder#stream()</code>. Here, you must define the desired schema via the <code class="language-plaintext highlighter-rouge">Consumed.with()</code> parameter for reading the topic’s data:</p>
<div class="language-java highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="nc">StreamsBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="k">new</span> <span class="nc">StreamsBuilder</span><span class="o">();</span>
<span class="nc">KStream</span><span class="o"><</span><span class="nc">String</span><span class="o">,</span> <span class="nc">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span>
<span class="n">builder</span><span class="o">.</span><span class="na">stream</span><span class="o">(</span><span class="s">"input-topic"</span><span class="o">,</span> <span class="nc">Consumed</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="nc">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="nc">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">()));</span>
</code></pre></div></div>
<p>In KSQL you would do something like the following to read a topic as a <code class="language-plaintext highlighter-rouge">STREAM</code>. Here, you must define the desired schema by defining column names and types for reading the topic’s data:</p>
<div class="language-sql highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">CREATE</span> <span class="n">STREAM</span> <span class="n">myStream</span> <span class="p">(</span><span class="n">username</span> <span class="nb">VARCHAR</span><span class="p">,</span> <span class="k">location</span> <span class="nb">VARCHAR</span><span class="p">)</span>
<span class="k">WITH</span> <span class="p">(</span><span class="n">KAFKA_TOPIC</span><span class="o">=</span><span class="s1">'input-topic'</span><span class="p">,</span> <span class="n">VALUE_FORMAT</span><span class="o">=</span><span class="s1">'...'</span><span class="p">)</span>
</code></pre></div></div>
<h2 id="tables">Tables</h2>
<p>We now read the same topic into a table. First, we need to add schema information (schema-on-read). Second, we must convert the stream into a table. The table semantics in Kafka say that the resulting table must map every message key in the topic to the latest message value for that key.</p>
<p>Let’s use the first example from the beginning, where the resulting table tracks the latest location of each user:</p>
<p><img src="/assets/uploads/stream-table-animation-latestLocation.gif" alt="First use case example (table tracks latest values per key)" /></p>
<p>In Scala:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Scala analogy</span>
<span class="n">scala</span><span class="o">></span> <span class="k">val</span> <span class="nv">table</span> <span class="k">=</span> <span class="n">topic</span>
<span class="o">|</span> <span class="o">.</span><span class="py">map</span> <span class="o">{</span> <span class="nf">case</span> <span class="o">(</span><span class="n">k</span><span class="k">:</span> <span class="kt">Array</span><span class="o">[</span><span class="kt">Byte</span><span class="o">],</span> <span class="n">v</span><span class="k">:</span> <span class="kt">Array</span><span class="o">[</span><span class="kt">Byte</span><span class="o">])</span> <span class="k">=></span> <span class="k">new</span> <span class="nc">String</span><span class="o">(</span><span class="n">k</span><span class="o">)</span> <span class="o">-></span> <span class="k">new</span> <span class="nc">String</span><span class="o">(</span><span class="n">v</span><span class="o">)</span> <span class="o">}</span>
<span class="o">|</span> <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">_1</span><span class="o">)</span>
<span class="o">|</span> <span class="o">.</span><span class="py">map</span> <span class="o">{</span> <span class="nf">case</span> <span class="o">(</span><span class="n">k</span><span class="o">,</span> <span class="n">v</span><span class="o">)</span> <span class="k">=></span> <span class="o">(</span><span class="n">k</span><span class="o">,</span> <span class="nv">v</span><span class="o">.</span><span class="py">reduceLeft</span><span class="o">(</span> <span class="o">(</span><span class="n">aggV</span><span class="o">,</span> <span class="n">newV</span><span class="o">)</span> <span class="k">=></span> <span class="n">newV</span><span class="o">).</span><span class="py">_2</span><span class="o">)</span> <span class="o">}</span>
<span class="c1">// => table: scala.collection.immutable.Map[String,String] =</span>
<span class="c1">// Map(alice -> Berlin, bob -> Lima)</span>
</code></pre></div></div>
<p>Adding schema information is achieved by the first <code class="language-plaintext highlighter-rouge">map()</code> – just like in the stream example above. The stream-to-table conversion is achieved by an <em>aggregation</em> step (more on this later), which in the case represents a (stateless) UPSERT operation on the table: this is the <code class="language-plaintext highlighter-rouge">groupBy().map()</code> step that contains a per-key <code class="language-plaintext highlighter-rouge">reduceLeft()</code> operation. Aggregation means that, for each key, we are squashing many values into a single value. Note that this particular <code class="language-plaintext highlighter-rouge">reduceLeft()</code> aggregation is stateless – the previous value <code class="language-plaintext highlighter-rouge">aggV</code> is not used to compute the new, next aggregate for a given key.</p>
<p>What’s interesting with regards to the relation between streams and tables is that the above command to create the table is equivalent to the shorter variant below (think: <a href="https://en.wikipedia.org/wiki/Referential_transparency">referential transparency</a>), where we build the table directly from the stream, which allows us to skip the schema/type definition because the stream is already typed. We can see now that <em>a table is a derivation, an aggregation of a stream</em>:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Scala analogy, simplified</span>
<span class="n">scala</span><span class="o">></span> <span class="k">val</span> <span class="nv">table</span> <span class="k">=</span> <span class="n">stream</span>
<span class="o">|</span> <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">_1</span><span class="o">)</span>
<span class="o">|</span> <span class="o">.</span><span class="py">map</span> <span class="o">{</span> <span class="nf">case</span> <span class="o">(</span><span class="n">k</span><span class="o">,</span> <span class="n">v</span><span class="o">)</span> <span class="k">=></span> <span class="o">(</span><span class="n">k</span><span class="o">,</span> <span class="nv">v</span><span class="o">.</span><span class="py">reduceLeft</span><span class="o">(</span> <span class="o">(</span><span class="n">aggV</span><span class="o">,</span> <span class="n">newV</span><span class="o">)</span> <span class="k">=></span> <span class="n">newV</span><span class="o">).</span><span class="py">_2</span><span class="o">)</span> <span class="o">}</span>
<span class="c1">// => table: scala.collection.immutable.Map[String,String] =</span>
<span class="c1">// Map(alice -> Berlin, bob -> Lima)</span>
</code></pre></div></div>
<p>In Kafka Streams you’d normally use <code class="language-plaintext highlighter-rouge">StreamsBuilder#table()</code> to read a Kafka topic into a KTable with a simple 1-liner:</p>
<div class="language-java highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="nc">KTable</span><span class="o"><</span><span class="nc">String</span><span class="o">,</span> <span class="nc">String</span><span class="o">></span> <span class="n">table</span> <span class="o">=</span> <span class="n">builder</span><span class="o">.</span><span class="na">table</span><span class="o">(</span><span class="s">"input-topic"</span><span class="o">,</span> <span class="nc">Consumed</span><span class="o">.</span><span class="na">with</span><span class="o">(</span><span class="nc">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">(),</span> <span class="nc">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">()));</span>
</code></pre></div></div>
<p>But, for the sake of illustration, you can also read the topic into a KStream first, and then perform the same aggregation step as shown above explicitly to turn the KStream into a KTable.</p>
<div class="language-java highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="nc">KStream</span><span class="o"><</span><span class="nc">String</span><span class="o">,</span> <span class="nc">String</span><span class="o">></span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span>
<span class="nc">KTable</span><span class="o"><</span><span class="nc">String</span><span class="o">,</span> <span class="nc">String</span><span class="o">></span> <span class="n">table</span> <span class="o">=</span> <span class="n">stream</span>
<span class="o">.</span><span class="na">groupByKey</span><span class="o">()</span>
<span class="o">.</span><span class="na">reduce</span><span class="o">((</span><span class="n">aggV</span><span class="o">,</span> <span class="n">newV</span><span class="o">)</span> <span class="o">-></span> <span class="n">newV</span><span class="o">);</span>
</code></pre></div></div>
<p>In KSQL you would do something like the following to read a topic as a <code class="language-plaintext highlighter-rouge">TABLE</code>. Here, you must define the desired schema by defining column names and types for reading the topic’s data:</p>
<div class="language-sql highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">CREATE</span> <span class="k">TABLE</span> <span class="n">myTable</span> <span class="p">(</span><span class="n">username</span> <span class="nb">VARCHAR</span><span class="p">,</span> <span class="k">location</span> <span class="nb">VARCHAR</span><span class="p">)</span>
<span class="k">WITH</span> <span class="p">(</span><span class="n">KAFKA_TOPIC</span><span class="o">=</span><span class="s1">'input-topic'</span><span class="p">,</span> <span class="k">KEY</span><span class="o">=</span><span class="s1">'username'</span><span class="p">,</span> <span class="n">VALUE_FORMAT</span><span class="o">=</span><span class="s1">'...'</span><span class="p">)</span>
</code></pre></div></div>
<p>What does this mean? It means that a table is actually an <em>aggregated stream</em>, just like we said at the very beginning. We have seen this first-hand in the special case above where a table is created straight from a topic. But it turns out that this is actually the general case.</p>
<h1 id="tables-stand-on-the-shoulders-of-stream-giants">Tables Stand on the Shoulders of Stream Giants</h1>
<p>Conceptually, only the stream is a first-order data construct in Kafka. A table, on the other hand, is either (1) derived from an existing stream through per-key aggregation or (2) derived from an existing table, whose lineage can always be traced back to an aggregated stream (we might call the latter the tables’ “ur-stream”).</p>
<blockquote>
<p>Tables are often also described as being a <em>materialized view</em> of a stream. A view of a stream is nothing but an aggregation in this context.</p>
</blockquote>
<p>Of the two cases the more interesting one to discuss is (1), so let’s focus on that. And this probably means that I need to first clarify how aggregations work in Kafka.</p>
<h2 id="aggregations-in-kafka">Aggregations in Kafka</h2>
<p>Aggregations are one type of operation in stream processing. Other types are filters and joins, for example.</p>
<p>As we have learned in the beginning, data is represented as key-value pairs in Kafka. Now, the first characteristic of aggregations in Kafka is that all aggregations are computed <em>per key</em>. That’s why we must group a KStream prior to the actual aggregation step in Kafka Streams via <code class="language-plaintext highlighter-rouge">groupBy()</code> or <code class="language-plaintext highlighter-rouge">groupByKey()</code>. For the same reason we had to use <code class="language-plaintext highlighter-rouge">groupBy()</code> in the Scala illustrations above.</p>
<div class="note">
<strong>Partitions and message keys:</strong> An equally important facet of Kafka that I am ignoring in this article is that topics, streams, and tables are <em>partitioned</em>. Data is actually processed and aggregated per key, per partition. By default, messages/records are assigned to partitions based on their keys, so in practice the simplification of "aggregated per key" instead of the technically more correct "aggregated per key, per partition" is accurate. But if you are using custom partitioning assigners, then you must account for this in your processing logic.
</div>
<p>The second characteristic of aggregations in Kafka is that aggregations are continuously updated as soon as new data arrives in the input streams. Together with the per-key computation characteristics, this requires having a <em>table</em> and, more precisely, a <em>mutable table</em> as the output and thus the return type of aggregations. Previous values (aggregation results) for a key are continuously being overwritten with newer values. In both Kafka Streams and KSQL, aggregations always return a table.</p>
<p>Let’s go back to our second use case example, where we want to count the number of locations visited by each user in our example stream:</p>
<p><img src="/assets/uploads/stream-table-animation-numVisitedLocations.gif" alt="Second use case example (table tracks counts per key)" /></p>
<p>Counting is a type of aggregation. To do this we only need to replace the aggregation step of the previous section <code class="language-plaintext highlighter-rouge">.reduce((aggV, newV) -> newV)</code> with <code class="language-plaintext highlighter-rouge">.map { case (k, v) => (k, v.length) }</code> to perform the counting. Note how the return type is a table/map (and please ignore that, in the Scala code, the map is immutable because Scala defaults to immutable maps).</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Scala analogy</span>
<span class="n">scala</span><span class="o">></span> <span class="k">val</span> <span class="nv">visitedLocationsPerUser</span> <span class="k">=</span> <span class="n">stream</span>
<span class="o">|</span> <span class="o">.</span><span class="py">groupBy</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">_1</span><span class="o">)</span>
<span class="o">|</span> <span class="o">.</span><span class="py">map</span> <span class="o">{</span> <span class="nf">case</span> <span class="o">(</span><span class="n">k</span><span class="o">,</span> <span class="n">v</span><span class="o">)</span> <span class="k">=></span> <span class="o">(</span><span class="n">k</span><span class="o">,</span> <span class="nv">v</span><span class="o">.</span><span class="py">length</span><span class="o">)</span> <span class="o">}</span>
<span class="c1">// => visitedLocationsPerUser: scala.collection.immutable.Map[String,Int] =</span>
<span class="c1">// Map(alice -> 3, bob -> 2)</span>
</code></pre></div></div>
<p>The Kafka Streams equivalent of the Scala example above is:</p>
<div class="language-java highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="nc">KTable</span><span class="o"><</span><span class="nc">String</span><span class="o">,</span> <span class="nc">Long</span><span class="o">></span> <span class="n">visitedLocationsPerUser</span> <span class="o">=</span> <span class="n">stream</span>
<span class="o">.</span><span class="na">groupByKey</span><span class="o">()</span>
<span class="o">.</span><span class="na">count</span><span class="o">();</span>
</code></pre></div></div>
<p>In KSQL:</p>
<div class="language-sql highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">CREATE</span> <span class="k">TABLE</span> <span class="n">visitedLocationsPerUser</span> <span class="k">AS</span>
<span class="k">SELECT</span> <span class="n">username</span><span class="p">,</span> <span class="k">COUNT</span><span class="p">(</span><span class="o">*</span><span class="p">)</span>
<span class="k">FROM</span> <span class="n">myStream</span>
<span class="k">GROUP</span> <span class="k">BY</span> <span class="n">username</span><span class="p">;</span>
</code></pre></div></div>
<h2 id="tables-are-aggregated-streams-input-stream--table">Tables are Aggregated Streams (input stream → table)</h2>
<p>As we have seen above tables are aggregations of their input streams or, in short, tables are aggregated streams. Whenever you are performing an aggregation in Kafka Streams or KSQL, i.e. turning <em>N</em> input records into <em>1</em> output record, the result is always a table. In KSQL, such functions are aptly called aggregate functions, and with KSQL 5.0 you can also provide your own custom implementations via User Defined Aggregate Functions (UDAF).</p>
<p>The specifics of the aggregation step determine whether the table is trivially derived from a stream via stateless UPSERT semantics (table maps keys to their latest value in the stream, which is the aggregation type used when reading a Kafka topic straight into a table), via stateful counting of the number of values seen per key (see our last example), or more sophisticated aggregations such as summing, averaging, and so on. When using Kafka Streams and KSQL you have many options for aggregations, including windowed aggregations with tumbling windows, hopping windows, and session windows.</p>
<h2 id="tables-have-changelog-streams-table--output-stream">Tables have Changelog Streams (table → output stream)</h2>
<p>While a table is an aggregation of its input stream, it also has its own output stream! Similar to change data capture (CDC) in databases, every change or mutation of a table in Kafka is captured behind the scenes in an internally used stream of changes aptly called the table’s <em>changelog stream</em>. Many computations in Kafka Streams and KSQL are actually performed on a table’s changelog stream rather than directly on the table. This enables Kafka Streams and KSQL to, for example, correctly re-process historical data according to event-time processing semantics – remember, a stream represents the present and the past, whereas a table can only represent the present (or, more precisely, a snapshot in time).</p>
<div class="note">
<strong>Note:</strong> In Kafka Streams you can convert a table explicitly into its changelog stream via <code>KTable#toStream()</code>.
</div>
<p>Here is the first use case example, now with the table’s changelog stream being shown:</p>
<p><img src="/assets/uploads/stream-table-animation-latestLocation-changelog.gif" alt="First use case example (table tracks latest values per key, with changelog stream shown)" /></p>
<p>Note how the table’s changelog stream is a copy of the table’s input stream. That’s because of the nature of the corresponding aggregation function (UPSERT). And if you’re wondering “Wait, isn’t this 1:1 copying a waste of storage space?” – Kafka Streams and KSQL perform optimizations under the hood to minimize needless data copies and local/network IO. I ignore these optimizations in the diagram above to better illustrate what’s happening in principal.</p>
<p>And, lastly, the second use case example including changelog stream. Here, the table’s changelog stream is different from the table’s input stream because the aggregation function, which performs per-key counting, is different.</p>
<p><img src="/assets/uploads/stream-table-animation-numVisitedLocations-changelog.gif" alt="Second use case example (table tracks counts per key, with changelog stream shown)" /></p>
<p>But these internal changelog streams also have architectural and operational impacts. Changelog streams are continuously backed up and stored as topics in Kafka, and thereby part of the magic that enables elasticity and fault-tolerance in Kafka Streams and KSQL. That’s because they allow moving processing tasks across machines/VMs/containers without data loss and during live operations, regardless of whether the processing is stateless or stateful. A table is part of your application’s (Kafka Streams) or query’s (KSQL) state, hence it is mandatory for Kafka to ensure that it can move not just the processing code (this is easy) but also the processing state including tables across machines in a fast and reliable manner (this is much harder). Whenever a table needs to be moved from client machine A to B, what happens behind the scenes is that, at the new destination B, the table is reconstructed from its changelog stream in Kafka (server-side) to exactly how it was on machine A. We can see this in the last diagram above, where the “counting table” can be readily restored from its changelog stream without having to reprocess the input stream.</p>
<h2 id="the-stream-table-duality">The Stream-Table Duality</h2>
<p>The term <a href="https://docs.confluent.io/current/streams/concepts.html#duality-of-streams-and-tables">stream-table duality</a> refers to the above relationship between streams and tables. It means, for example, that you can turn a stream into a table into a stream into a table and so on. See Confluent’s blog post <a href="https://www.confluent.io/blog/introducing-kafka-streams-stream-processing-made-simple/">Introducing Kafka Streams: Stream Processing Made Simple</a> for further information.</p>
<h1 id="turning-the-database-inside-out">Turning the Database Inside-Out</h1>
<p>In addition to what we covered in the previous sections, you might have come across the article <a href="https://www.confluent.io/blog/turning-the-database-inside-out-with-apache-samza/">Turning the Database Inside-Out</a>, and now you might be wondering what’s the 10,000 feet view of all this? While I don’t want to go into much detail here, let me briefly juxtapose the world of Kafka and stream processing with the world of databases. Caveat emptor: black-and-white simplifications ahead.</p>
<p>In <strong>databases</strong>, the first-order construct is the <em>table</em>. This is what you work with. “Streams” also exist in databases, for example in the form of <a href="https://dev.mysql.com/doc/refman/5.7/en/binary-log.html">MySQL’s binlog</a> or <a href="https://docs.oracle.com/en/database/oracle/oracle-database/18/admin/managing-the-redo-log.html">Oracle’s Redo Log</a>, but they are typically hidden from you in the sense that you do not interact with them directly. A database knows about the present, but it does not know about the past (if you need the past, fetch your backup tapes which, haha, are hardware streams).</p>
<p>In <strong>Kafka and stream processing</strong>, the first-order construct is the <em>stream</em>. Tables are derivations of streams, as we have seen above. Kafka knows about the present but also about the past. As an example of anecdotal evidence, <a href="https://open.nytimes.com/publishing-with-apache-kafka-at-the-new-york-times-7f0e3b7d2077">The New York Times store all articles ever published</a> – 160 years of journalism going back to the 1850’s – in Kafka as the source of truth.</p>
<p><span class="pullquote-right">A database thinks table first, stream second. Kafka thinks stream first, table second.</span>
In short:
A database thinks table first, stream second. Kafka thinks stream first, table second.
That said, the Kafka community has realized that most streaming use cases in practice require both streams <em>and</em> tables – even the infamous yet simple WordCount, which aggregates a stream of text lines into a table of word counts, like our second use case example above. Hence Kafka helps you to bridge the worlds of stream processing and databases by providing native support for streams and tables via Kafka Streams and KSQL in order to save you from a lot of DIY pain (and pager alerts). We might call Kafka and the type of streaming platform it represents therefore <a href="https://yokota.blog/2018/03/05/stream-relational-processing-platforms/">stream-relational</a> rather than stream-only.</p>
<h1 id="wrapping-up">Wrapping Up</h1>
<p>I hope you find these explanations useful to better understand streams and tables in Kafka and in stream processing at large. Now that we have finished our closer look, you might want to go back to the beginning of the article to re-read the “Streams and Tables in Plain English” and “Streams and Tables in Kafka English” sections one more time.</p>
<p>If this article made you curious to try out stream-relational processing with Kafka, Kafka Streams, and KSQL, you might want to continue with:</p>
<ul>
<li>Learning how to use <a href="https://github.com/confluentinc/ksql">KSQL</a>, the streaming SQL engine for Kafka, to process your Kafka data without writing any programming code. That’s what I would recommend as your starting point, particularly if you are new to Kafka or stream processing, as you should get up and running in a matter of minutes. There’s also a cool KSQL clickstream demo (including a Docker variant) where you can play with a Kafka, KSQL, Elasticsearch, Grafana setup to drive a real-time dashboard.</li>
<li>Learning how to build Java or Scala applications for stream processing with the <a href="https://kafka.apache.org/documentation/streams/">Kafka Streams API</a>.</li>
<li>And yes, you can of course combine the two, e.g. you can start processing your data with KSQL, then continue with Kafka Streams, and then follow-up again with KSQL.</li>
</ul>
<p>Regardless of whether you are using Kafka Streams or KSQL, thanks to Kafka you’ll benefit from elastic, scalable, fault-tolerant, distributed stream processing that runs everywhere (containers, VMs, machines, locally, on-prem, cloud, you name it). Just saying in case this isn’t obvious. :-)</p>
<p>Lastly, I titled this article as Part 1 of Streams and Tables. And while I already have ideas for Part 2, I’d appreciate questions or suggestions on what I could cover next. What do you want to learn more about? Let me know in the comments below or drop me an email!</p>In this article, perhaps the first in a mini-series, I want to explain the concepts of streams and tables in stream processing and, specifically, in Apache Kafka. Hopefully, you will walk away with both a better theoretical understanding but also more tangible insights and ideas that will help you solve your current or next practical use case better, faster, or both.Integrating Kafka and Spark Streaming: Code Examples and State of the Game2014-10-01T16:51:00+02:002014-10-01T16:51:00+02:00https://www.michael-noll.com/blog/2014/10/01/kafka-spark-streaming-integration-example-tutorial<p><a href="https://spark.apache.org/streaming/">Spark Streaming</a> has been getting some attention lately as a real-time data
processing tool, often mentioned alongside <a href="http://storm.apache.org/">Apache Storm</a>. If you ask me, no real-time data
processing tool is complete without Kafka integration (smile), hence I added an example Spark Streaming application to
<a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> that demonstrates how to read from Kafka and write
to Kafka, using <a href="http://avro.apache.org/">Avro</a> as the data format and
<a href="https://github.com/twitter/bijection">Twitter Bijection</a> for handling the data serialization.
In this post I will explain this Spark Streaming example in further detail and also shed some light on the current state
of Kafka integration in Spark Streaming. All this with the disclaimer that this happens to be my first experiment with
Spark Streaming.</p>
<!-- more -->
<ul id="markdown-toc">
<li><a href="#what-is-spark-streaming" id="markdown-toc-what-is-spark-streaming">What is Spark Streaming?</a> <ul>
<li><a href="#spark-streaming-vs-apache-storm" id="markdown-toc-spark-streaming-vs-apache-storm">Spark Streaming vs. Apache Storm</a></li>
</ul>
</li>
<li><a href="#excursus-machines-cores-executors-tasks-and-receivers-in-spark" id="markdown-toc-excursus-machines-cores-executors-tasks-and-receivers-in-spark">Excursus: Machines, cores, executors, tasks, and receivers in Spark</a></li>
<li><a href="#integrating-kafka-with-spark-streaming" id="markdown-toc-integrating-kafka-with-spark-streaming">Integrating Kafka with Spark Streaming</a> <ul>
<li><a href="#overview" id="markdown-toc-overview">Overview</a></li>
<li><a href="#primer-on-topics-partitions-and-parallelism-in-kafka" id="markdown-toc-primer-on-topics-partitions-and-parallelism-in-kafka">Primer on topics, partitions, and parallelism in Kafka</a></li>
<li><a href="#reading-from-kafka" id="markdown-toc-reading-from-kafka">Reading from Kafka</a> <ul>
<li><a href="#read-parallelism-in-spark-streaming" id="markdown-toc-read-parallelism-in-spark-streaming">Read parallelism in Spark Streaming</a> <ul>
<li><a href="#option-1-controlling-the-number-of-input-dstreams" id="markdown-toc-option-1-controlling-the-number-of-input-dstreams">Option 1: Controlling the number of input DStreams</a></li>
<li><a href="#option-2-controlling-the-number-of-consumer-threads-per-input-dstream" id="markdown-toc-option-2-controlling-the-number-of-consumer-threads-per-input-dstream">Option 2: Controlling the number of consumer threads per input DStream</a></li>
<li><a href="#combining-options-1-and-2" id="markdown-toc-combining-options-1-and-2">Combining options 1 and 2</a></li>
</ul>
</li>
<li><a href="#downstream-processing-parallelism-in-spark-streaming" id="markdown-toc-downstream-processing-parallelism-in-spark-streaming">Downstream processing parallelism in Spark Streaming</a></li>
</ul>
</li>
<li><a href="#writing-to-kafka" id="markdown-toc-writing-to-kafka">Writing to Kafka</a></li>
<li><a href="#complete-example" id="markdown-toc-complete-example">Complete example</a></li>
</ul>
</li>
<li><a href="#known-issues-in-spark-streaming" id="markdown-toc-known-issues-in-spark-streaming">Known issues in Spark Streaming</a></li>
<li><a href="#spark-tips-and-tricks" id="markdown-toc-spark-tips-and-tricks">Spark tips and tricks</a> <ul>
<li><a href="#general" id="markdown-toc-general">General</a></li>
<li><a href="#kafka-integration" id="markdown-toc-kafka-integration">Kafka integration</a></li>
<li><a href="#testing" id="markdown-toc-testing">Testing</a></li>
<li><a href="#performance-tuning" id="markdown-toc-performance-tuning">Performance tuning</a></li>
</ul>
</li>
<li><a href="#wrapping-up" id="markdown-toc-wrapping-up">Wrapping up</a></li>
<li><a href="#references" id="markdown-toc-references">References</a></li>
</ul>
<div class="note">
<strong>
The Spark Streaming example code is available at
<a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> on GitHub.
And yes, the project's name might now be a bit misleading. :-)
</strong>
</div>
<h1 id="what-is-spark-streaming">What is Spark Streaming?</h1>
<p><a href="http://spark.apache.org/streaming/">Spark Streaming</a> is a sub-project of <a href="http://spark.apache.org/">Apache Spark</a>.
Spark is a batch processing platform similar to Apache Hadoop, and Spark Streaming is a real-time processing tool
that runs on top of the Spark engine.</p>
<h2 id="spark-streaming-vs-apache-storm">Spark Streaming vs. Apache Storm</h2>
<p>In terms of use cases Spark Streaming is closely related to <a href="http://storm.apache.org/">Apache Storm</a>, which is
arguably today’s most popular real-time processing platform for Big Data. Bobby Evans and Tom Graves of Yahoo!
Engineering recently gave a talk on
<a href="http://yahoohadoop.tumblr.com/post/98213421641/storm-and-spark-at-yahoo-why-chose-one-over-the-other">Spark and Storm at Yahoo!</a>,
in which they compare the two platforms and also cover the question of when and why choosing one over the other.
Similarly, P. Taylor Goetz of HortonWorks shared a slide deck titled
<a href="http://www.slideshare.net/ptgoetz/apache-storm-vs-spark-streaming">Apache Storm and Spark Streaming Compared</a>.</p>
<p>Here’s my personal, very brief comparison: Storm has higher industry adoption and better production stability compared
to Spark Streaming. Spark on the other hand has a more expressive, higher level API than Storm, which is arguably more
pleasant to use, at least if you write your Spark applications in Scala (I prefer the Spark API, too). But don’t trust
my word, please do check out the talks/decks above yourself.</p>
<p>Both Spark and Storm are top-level Apache projects, and vendors have begun to integrate either or both tools into their
commercial offerings, e.g. HortonWorks (<a href="http://hortonworks.com/hadoop/storm/">Storm</a>,
<a href="http://hortonworks.com/hadoop/spark/">Spark</a>) and Cloudera
(<a href="http://www.cloudera.com/content/cloudera/en/products-and-services/cdh/spark.html">Spark</a>).</p>
<h1 id="excursus-machines-cores-executors-tasks-and-receivers-in-spark">Excursus: Machines, cores, executors, tasks, and receivers in Spark</h1>
<p>The subsequent sections of this article talk a lot about parallelism in Spark and in Kafka. You need at least a basic
understanding of some Spark terminology to be able to follow the discussion in those sections.</p>
<ul>
<li>A Spark <strong>cluster</strong> contains 1+ worker nodes aka slave machines (simplified view; I exclude pieces like cluster
managers here.)</li>
<li>A <strong>worker node</strong> can run 1+ executors.</li>
<li>An <strong>executor</strong> is a process launched for an application on a worker node, which runs tasks and keeps data in memory
or disk storage across them. Each application has its own executors. An executor has a certain amount of cores aka
“slots” available to run tasks assigned to it.</li>
<li>A <strong>task</strong> is a unit of work that will be sent to one executor. That is, it runs (part of) the actual computation of
your application. The <code class="language-plaintext highlighter-rouge">SparkContext</code> sends those tasks for the executors to run. Each task occupies one slot aka
core in the parent executor.</li>
<li>A <strong>receiver</strong>
(<a href="http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.streaming.receiver.Receiver">API</a>,
<a href="http://spark.apache.org/docs/latest/streaming-custom-receivers.html">docs</a>)
is run within an executor as a long-running task. Each receiver is responsible for exactly one so-called
<em>input DStream</em> (e.g. an input stream for reading from Kafka), and each receiver – and thus input DStream – occupies
one core/slot.</li>
<li>An <strong>input DStream</strong>: an <em>input</em> DStream is a special DStream that connects Spark Streaming to external data sources
for reading input data. For each external data source (e.g. Kafka) you need one such input DStream implementation.
Once Spark Streaming is “connected” to an external data source via such input DStreams, any subsequent DStream
transformations will create “normal” DStreams.</li>
</ul>
<p>In Spark’s execution model, each application gets its own executors, which stay up for the duration of the whole
application and run 1+ tasks in multiple threads. This isolation approach is similar to Storm’s model of execution.
This architecture becomes more complicated once you introduce cluster managers like YARN or Mesos, which I do not cover
here. See <a href="http://spark.apache.org/docs/latest/cluster-overview.html">Cluster Overview</a> in the Spark docs for further
details.</p>
<h1 id="integrating-kafka-with-spark-streaming">Integrating Kafka with Spark Streaming</h1>
<h2 id="overview">Overview</h2>
<p>In short, Spark Streaming supports Kafka but there are still some rough edges.</p>
<p>A good starting point for me has been the
<a href="https://github.com/apache/spark/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala">KafkaWordCount</a>
example in the Spark code base
(<strong>Update 2015-03-31:</strong> see also
<a href="https://github.com/apache/spark/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala">DirectKafkaWordCount</a>).
When I read this code, however, there were still a couple of open questions left.</p>
<p>Notably I wanted to understand how to:</p>
<ul>
<li>Read from Kafka <em>in parallel</em>. In Kafka, a topic can have <em>N</em> partitions, and ideally we’d like to parallelize
reading from those <em>N</em> partitions. This is what the
<a href="https://github.com/apache/incubator-storm/tree/master/external/storm-kafka">Kafka spout in Storm</a> does.</li>
<li>Write to Kafka from a Spark Streaming application, also, <em>in parallel</em>.</li>
</ul>
<p>On top of those questions I also ran into several known issues in Spark and/or Spark Streaming, most of which have been
discussed in the Spark mailing list. I’ll summarize the current state and known issues of the Kafka integration further
down below.</p>
<h2 id="primer-on-topics-partitions-and-parallelism-in-kafka">Primer on topics, partitions, and parallelism in Kafka</h2>
<p><em>For details see my articles</em>
<em><a href="/blog/2014/08/18/apache-kafka-training-deck-and-tutorial/">Apache Kafka 0.8 Training Deck and Tutorial</a></em>
<em>and</em>
<em><a href="/blog/2013/03/13/running-a-multi-broker-apache-kafka-cluster-on-a-single-node/">Running a Multi-Broker Apache Kafka 0.8 Cluster on a Single Node</a>.</em></p>
<p>Kafka stores data in <em>topics</em>, with each topic consisting of a configurable number of <em>partitions</em>. The number of
partitions of a topic is very important for performance considerations as this number is an <em>upper bound on the</em>
<em>consumer parallelism</em>: if a topic has <em>N</em> partitions, then your application can only consume this topic with a maximum
of <em>N</em> threads in parallel. (At least this is the case when you use Kafka’s built-in Scala/Java consumer API.)</p>
<p>When I say “application” I should rather say <em>consumer group</em> in Kafka’s terminology. A consumer group, identified by
a string of your choosing, is the cluster-wide identifier for a logical consumer application. All consumers that are
part of the same consumer group share the burden of reading from a given Kafka topic, and only a maximum of <em>N</em> (=
number of partitions) threads across all the consumers in the same group will be able to read from the topic. Any
excess threads will sit idle.</p>
<div class="note">
<strong>Multiple Kafka consumer groups can be run in parallel:</strong> Of course you can run multiple, independent logical consumer applications against the same Kafka topic. Here, each logical application will run its consumer threads under a unique consumer group id. Each application can then also use different read parallelisms (see below). When I am talking about the various ways to configure read parallelism in the following sections, then I am referring to the settings of a <em>single</em> one of these logical consumer applications.
</div>
<p>Here are some simplified examples.</p>
<ul>
<li>Your application uses the consumer group id “terran” to read from a Kafka topic “zerg.hydra” that has
<strong>10 partitions</strong>.
If you configure your application to consume the topic with only <strong>1</strong> thread, then this single thread will read data
from all 10 partitions.</li>
<li>Same as above, but this time you configure <strong>5</strong> consumer threads. Here, each thread will read from 2 partitions.</li>
<li>Same as above, but this time you configure <strong>10</strong> consumer threads. Here, each thread will read from a single
partition.</li>
<li>Same as above, but this time you configure <strong>14</strong> consumer threads. Here, 10 of the 14 threads will read from a
single partition each, and the remaining 4 threads will be idle.</li>
</ul>
<p>Let’s introduce some real-world complexity in this simple picture – the <em>rebalancing</em> event in Kafka. Rebalancing is
a lifecycle event in Kafka that occurs when consumers join or leave a consumer group (there are more conditions that
trigger rebalancing but these are not important in this context; see my
<a href="/blog/2014/08/18/apache-kafka-training-deck-and-tutorial/">Kafka training deck</a> for details on rebalancing).</p>
<ul>
<li>Your application uses the consumer group id “terran” and starts consuming with 1 thread. This thread will read from
all 10 partitions. During runtime, you’ll increase the number of threads from 1 to 14. That is, there is suddenly
a change of parallelism for the same consumer group. This triggers <em>rebalancing</em> in Kafka. Once rebalancing
completes, you will have 10 of 14 threads consuming from a single partition each, and the 4 remaining threads will be
idle. And as you might have guessed, the initial thread will now read from only one partition and will no longer see
data from the other nine.</li>
</ul>
<p>We have now a basic understanding of topics, partitions, and the number of partitions as an upper bound for the
parallelism when reading from Kafka. But what are the resulting implications for an application – such as a Spark
Streaming job or Storm topology – that reads its input data from Kafka?</p>
<ol>
<li><strong>Read parallelism:</strong> You typically want to read from all <em>N</em> partitions of a Kafka topic in parallel by consuming
with <em>N</em> threads. And depending on the data volume you want to spread those threads across different NICs, which
typically means across different machines. In Storm, this is achieved by setting the parallelism of the
<a href="https://github.com/apache/storm/tree/master/external/storm-kafka">Kafka spout</a> to <em>N</em> via
<code class="language-plaintext highlighter-rouge">TopologyBuilder#setSpout()</code>. The Spark equivalent is a bit trickier, and I will describe how to do this in further
detail below.</li>
<li><strong>Downstream processing parallelism:</strong> Once retrieved from Kafka you want to process the data in parallel.
Depending on your use case this level of parallelism must be different from the read parallelism. If your use case
is CPU-bound, for instance, you want to have many more processing threads than read threads; this is achieved by
shuffling or “fanning out” the data via the network from the few read threads to the many processing threads. Hence
you pay for the access to more cores with increased network communication, serialization overhead, etc. In Storm,
you perform such a shuffling via a
<a href="https://storm.apache.org/documentation/Concepts.html">shuffle grouping</a> from the Kafka spout to the next downstream
bolt. The Spark equivalent is the
<a href="https://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#transformations-on-dstreams">repartition</a>
transformation on DStreams.</li>
</ol>
<p>The important takeaway is that it is possible – and often desired – to decouple the level of parallelisms for
<em>reading from Kafka</em> and for <em>processing the data once read</em>. In the next sections I will describe the various options
you have at your disposal to configure read parallelism and downstream processing parallelism in Spark Streaming.</p>
<h2 id="reading-from-kafka">Reading from Kafka</h2>
<h3 id="read-parallelism-in-spark-streaming">Read parallelism in Spark Streaming</h3>
<p>Like Kafka, Spark Streaming has the concept of <em>partitions</em>. It is important to understand that Kafka’s per-topic
partitions are not correlated to the partitions of
<a href="http://spark.apache.org/docs/1.1.0/programming-guide.html">RDDs in Spark</a>.</p>
<p>The <a href="https://github.com/apache/spark/blob/master/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala">KafkaInputDStream</a>
of Spark Streaming – aka its Kafka “connector” – uses Kafka’s
<a href="http://kafka.apache.org/documentation.html#highlevelconsumerapi">high-level consumer API</a>, which means you have two
control knobs in Spark that determine read parallelism for Kafka:</p>
<ol>
<li><strong>The number of input DStreams.</strong> Because Spark will run one receiver (= task) per input DStream, this means using
multiple input DStreams will parallelize the read operations across multiple cores and thus, hopefully, across
multiple machines and thereby NICs.</li>
<li><strong>The number of consumer threads per input DStream.</strong> Here, the same receiver (= task) will run multiple threads.
That is, read operations will happen in parallel but on the same core/machine/NIC.</li>
</ol>
<p>For practical purposes option 1 is the preferred.</p>
<p>Why is that? First and foremost because reading from Kafka is
normally network/NIC limited, i.e. you typically do not increase read-throughput by running more threads <em>on the same</em>
<em>machine</em>. In other words, it is rare though possible that reading from Kafka runs into CPU bottlenecks. Second, if
you go with option 2 then multiple threads will be competing for the lock to push data into so-called <em>blocks</em> (the <code class="language-plaintext highlighter-rouge">+=</code>
method of <code class="language-plaintext highlighter-rouge">BlockGenerator</code> that is used behind the scenes is <code class="language-plaintext highlighter-rouge">synchronized</code> on the block generator instance).</p>
<div class="note">
<strong>Number of partitions of the RDDs created by the input DStreams:</strong> The <tt>KafkaInputDStream</tt> will store individual messages received from Kafka into so-called <em>blocks</em>. From what I understand, a new block is generated every <a href="http://spark.apache.org/docs/latest/configuration.html#spark-streaming">spark.streaming.blockInterval</a> milliseconds, and each block is turned into a partition of the RDD that will eventually be created by the DStream. If this assumption of mine is true, then the number of partitions in the RDDs created by <tt>KafkaInputDStream</tt> is determined by <tt>batchInterval / spark.streaming.blockInterval</tt>, where <tt>batchInterval</tt> is the time interval at which streaming data will be divided into batches (set via a constructor parameter of <tt>StreamingContext</tt>). For example, if the batch interval is 2 seconds (default) and the block interval is 200ms (default), your RDD will contain 10 partitions. Please correct me if I'm mistaken.
</div>
<h4 id="option-1-controlling-the-number-of-input-dstreams">Option 1: Controlling the number of input DStreams</h4>
<p>The example below is taken from the
<a href="https://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#reducing-the-processing-time-of-each-batch">Spark Streaming Programming Guide</a>.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">ssc</span><span class="k">:</span> <span class="kt">StreamingContext</span> <span class="o">=</span> <span class="o">???</span> <span class="c1">// ignore for now</span>
<span class="k">val</span> <span class="nv">kafkaParams</span><span class="k">:</span> <span class="kt">Map</span><span class="o">[</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span><span class="s">"group.id"</span> <span class="o">-></span> <span class="s">"terran"</span><span class="o">,</span> <span class="cm">/* ignore rest */</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">numInputDStreams</span> <span class="k">=</span> <span class="mi">5</span>
<span class="k">val</span> <span class="nv">kafkaDStreams</span> <span class="k">=</span> <span class="o">(</span><span class="mi">1</span> <span class="n">to</span> <span class="n">numInputDStreams</span><span class="o">).</span><span class="py">map</span> <span class="o">{</span> <span class="k">_</span> <span class="k">=></span> <span class="nv">KafkaUtils</span><span class="o">.</span><span class="py">createStream</span><span class="o">(...)</span> <span class="o">}</span>
</code></pre></div></div>
<p>In this example we create five input DStreams, thus spreading the burden of reading from Kafka across five cores and,
hopefully, five machines/NICs. (I say “hopefully” because I am not certain whether Spark Streaming task placement
policy will try to place receivers on different machines.) All input DStreams are part of the “terran” consumer group,
and the Kafka API will ensure that these five input DStreams a) will see all available data for the topic because it
assigns each partition of the topic to an input DStream and b) will not see overlapping data because each partition is
assigned to only one input DStream at a time. In other words, this setup of “collaborating” input DStreams works
because of the consumer group behavior provided by the Kafka API, which is used behind the scenes by
<code class="language-plaintext highlighter-rouge">KafkaInputDStream</code>.</p>
<p>What I have not shown in the example is how many threads are created <em>per input DStream</em>, which is done via parameters
to the <code class="language-plaintext highlighter-rouge">KafkaUtils.createStream</code> method (the actual input topic(s) are also specified as parameters of this method).
We will do this in the next section.</p>
<p>But before we continue let me highlight several known issues with this setup and with Spark Streaming in particular,
which are caused on the one hand by current limitations of Spark in general and on the other hand by the current
implementation of the Kafka input DStream in particular:</p>
<blockquote>
<p>[When you use the multi-input-stream approach I described above, then] those consumers operate in one [Kafka] consumer group, and they try to decide which consumer consumes which partitions. And it may just fail to do syncpartitionrebalance, and then you have only a few consumers really consuming. To mitigate this problem, you can set rebalance retries very high, and pray it helps.</p>
<p>Then arises yet another “feature” — if your receiver dies (OOM, hardware failure), you just stop receiving from Kafka!
(<a href="http://markmail.org/message/257a5l3oqyftsjxj">source</a>)</p>
</blockquote>
<p>The “stop receiving from Kafka” issue requires
<a href="http://apache-spark-user-list.1001560.n3.nabble.com/spark-streaming-and-the-spark-shell-tp3347.html">some explanation</a>.
Currently, when you start your streaming application
via <code class="language-plaintext highlighter-rouge">ssc.start()</code> the processing starts and continues indefinitely – even if the input data source (e.g. Kafka) becomes
unavailable. That is, streams are not able to detect if they have lost connection to the upstream data source and
thus cannot react to this event, e.g. by reconnecting or by stopping the execution. Similarly, if you lose a receiver
that reads from the data source, then
<a href="http://apache-spark-user-list.1001560.n3.nabble.com/spark-streaming-and-the-spark-shell-tp3347p3375.html">your streaming application will generate empty RDDs</a>.</p>
<p>This is a pretty unfortunate situation. One crude workaround is to restart your streaming application whenever it runs
into an upstream data source failure or a receiver failure. This workaround may not help you though if your use case
requires you to set the Kafka configuration option <code class="language-plaintext highlighter-rouge">auto.offset.reset</code> to “smallest” – because of a known bug in
Spark Streaming the resulting behavior of your streaming application may not be what you want. See the section on
<em>Known issues in Spark Streaming</em> below for further details.</p>
<h4 id="option-2-controlling-the-number-of-consumer-threads-per-input-dstream">Option 2: Controlling the number of consumer threads per input DStream</h4>
<p>In this example we create a <em>single</em> input DStream that is configured to run three consumer threads – in the same
receiver/task and thus on the same core/machine/NIC – to read from the Kafka topic “zerg.hydra”.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">ssc</span><span class="k">:</span> <span class="kt">StreamingContext</span> <span class="o">=</span> <span class="o">???</span> <span class="c1">// ignore for now</span>
<span class="k">val</span> <span class="nv">kafkaParams</span><span class="k">:</span> <span class="kt">Map</span><span class="o">[</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span><span class="s">"group.id"</span> <span class="o">-></span> <span class="s">"terran"</span><span class="o">,</span> <span class="o">...)</span>
<span class="k">val</span> <span class="nv">consumerThreadsPerInputDstream</span> <span class="k">=</span> <span class="mi">3</span>
<span class="k">val</span> <span class="nv">topics</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span><span class="s">"zerg.hydra"</span> <span class="o">-></span> <span class="n">consumerThreadsPerInputDstream</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">stream</span> <span class="k">=</span> <span class="nv">KafkaUtils</span><span class="o">.</span><span class="py">createStream</span><span class="o">(</span><span class="n">ssc</span><span class="o">,</span> <span class="n">kafkaParams</span><span class="o">,</span> <span class="n">topics</span><span class="o">,</span> <span class="o">...)</span>
</code></pre></div></div>
<p>The <code class="language-plaintext highlighter-rouge">KafkaUtils.createStream</code> method is overloaded, so there are a few different method signatures. In this example
we pick the Scala variant that gives us the most control.</p>
<h4 id="combining-options-1-and-2">Combining options 1 and 2</h4>
<p>Here is a more complete example that combines the previous two techniques:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">ssc</span><span class="k">:</span> <span class="kt">StreamingContext</span> <span class="o">=</span> <span class="o">???</span>
<span class="k">val</span> <span class="nv">kafkaParams</span><span class="k">:</span> <span class="kt">Map</span><span class="o">[</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span><span class="s">"group.id"</span> <span class="o">-></span> <span class="s">"terran"</span><span class="o">,</span> <span class="o">...)</span>
<span class="k">val</span> <span class="nv">numDStreams</span> <span class="k">=</span> <span class="mi">5</span>
<span class="k">val</span> <span class="nv">topics</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span><span class="s">"zerg.hydra"</span> <span class="o">-></span> <span class="mi">1</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">kafkaDStreams</span> <span class="k">=</span> <span class="o">(</span><span class="mi">1</span> <span class="n">to</span> <span class="n">numDStreams</span><span class="o">).</span><span class="py">map</span> <span class="o">{</span> <span class="k">_</span> <span class="k">=></span>
<span class="nv">KafkaUtils</span><span class="o">.</span><span class="py">createStream</span><span class="o">(</span><span class="n">ssc</span><span class="o">,</span> <span class="n">kafkaParams</span><span class="o">,</span> <span class="n">topics</span><span class="o">,</span> <span class="o">...)</span>
<span class="o">}</span>
</code></pre></div></div>
<p>We are creating five input DStreams, each of which will run a single consumer thread. If the input topic “zerg.hydra”
has five partitions (or less), then this is normally the best way to parallelize read operations if you care primarily
about maximizing throughput.</p>
<h3 id="downstream-processing-parallelism-in-spark-streaming">Downstream processing parallelism in Spark Streaming</h3>
<p>In the previous sections we covered parallelizing reads from Kafka. Now we can tackle parallelizing the downstream
data processing in Spark. Here, you must keep in mind how Spark itself parallelizes its processing. Like Kafka,
Spark ties the parallelism to the number of (RDD) partitions by running
<a href="http://spark.apache.org/docs/1.1.0/programming-guide.html#resilient-distributed-datasets-rdds"><em>one task per RDD partition</em></a>
(sometimes partitions are still called “slices” in the docs).</p>
<div class="note">
<strong>Just like any Spark application:</strong> Once a Spark Streaming application has received its input data, any
further processing is identical to non-streaming Spark applications. That is, you use exactly the same tools and
patterns to scale your application as you would for "normal" Spark data flows. See <a href="https://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#level-of-parallelism-in-data-processing">Level of Parallelism in Data Processing</a>.
</div>
<p>This gives us two control knobs:</p>
<ol>
<li><strong>The number of input DStreams</strong>, i.e. what we receive as a result of the previous sections on read parallelism.
This is our starting point, which we can either take as-is or modify with the next option.</li>
<li><strong>The</strong>
<strong><a href="https://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#transformations-on-dstreams">repartition</a></strong>
<strong>DStream transformation.</strong> It returns a new DStream with an increased or decreased level <em>N</em> of parallelism. Each
RDD in the returned DStream has exactly <em>N</em> partitions. DStreams are a continuous series of RDDs, and behind the
scenes <code class="language-plaintext highlighter-rouge">DStream.repartition</code> calls <code class="language-plaintext highlighter-rouge">RDD.repartition</code>. The latter “reshuffles the data in the RDD randomly to create
either more or fewer partitions and balance it across them. This always shuffles all data over the network.” In
other words, <code class="language-plaintext highlighter-rouge">DStream.repartition</code> is very similar to Storm’s
<a href="https://storm.apache.org/documentation/Concepts.html">shuffle grouping</a>.</li>
</ol>
<p>Hence <code class="language-plaintext highlighter-rouge">repartition</code> is our primary means to decouple read parallelism from processing parallelism. It allows us to
set the number of processing tasks and thus the number of cores that will be used for the processing. Indirectly, we
also influence the number of machines/NICs that will be involved.</p>
<p>A related DStream transformation is
<a href="https://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#transformations-on-dstreams">union</a>.
(This method also exists for <code class="language-plaintext highlighter-rouge">StreamingContext</code>, where it returns the unified DStream from multiple DStreams of the same
type and same slide duration. Most likely you would use the <code class="language-plaintext highlighter-rouge">StreamingContext</code> variant.) A <code class="language-plaintext highlighter-rouge">union</code> will return a
<code class="language-plaintext highlighter-rouge">UnionDStream</code> backed by a <code class="language-plaintext highlighter-rouge">UnionRDD</code>. A <code class="language-plaintext highlighter-rouge">UnionRDD</code> is comprised of all the partitions of the RDDs being unified, i.e.
if you unite 3 RDDs with 10 partitions each, then your union RDD instance will contain 30 partitions. In other words,
<code class="language-plaintext highlighter-rouge">union</code> will squash multiple DStreams into a single DStream/RDD, but it will not change the level of parallelism.
Whether you need to use <code class="language-plaintext highlighter-rouge">union</code> or not depends on whether your use case requires information from all Kafka partitions
“in one place”, so it’s primarily because of semantic requirements. One such example is when you need to perform a
(global) count of distinct elements.</p>
<div class="note">
Note: <a href="http://apache-spark-user-list.1001560.n3.nabble.com/Union-of-2-RDD-s-only-returns-the-first-one-tp766p5089.html">RDDs are not ordered.</a> So when you <tt>union</tt> RDDs, then the resulting RDD itself will not have a well-defined ordering either. If you need ordering <tt>sort</tt> the RDD.
</div>
<p>Your use case will determine which knobs and which combination thereof you need to use. Let’s say your use case is
CPU-bound. Here, you may want to consume the Kafka topic “zerg.hydra” (which has five Kafka partitions) with a read
parallelism of 5 – i.e. 5 receivers with 1 consumer thread each – but bump up the processing parallelism to 20:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">ssc</span><span class="k">:</span> <span class="kt">StreamingContext</span> <span class="o">=</span> <span class="o">???</span>
<span class="k">val</span> <span class="nv">kafkaParams</span><span class="k">:</span> <span class="kt">Map</span><span class="o">[</span><span class="kt">String</span>, <span class="kt">String</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span><span class="s">"group.id"</span> <span class="o">-></span> <span class="s">"terran"</span><span class="o">,</span> <span class="o">...)</span>
<span class="k">val</span> <span class="nv">readParallelism</span> <span class="k">=</span> <span class="mi">5</span>
<span class="k">val</span> <span class="nv">topics</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span><span class="s">"zerg.hydra"</span> <span class="o">-></span> <span class="mi">1</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">kafkaDStreams</span> <span class="k">=</span> <span class="o">(</span><span class="mi">1</span> <span class="n">to</span> <span class="n">readParallelism</span><span class="o">).</span><span class="py">map</span> <span class="o">{</span> <span class="k">_</span> <span class="k">=></span>
<span class="nv">KafkaUtils</span><span class="o">.</span><span class="py">createStream</span><span class="o">(</span><span class="n">ssc</span><span class="o">,</span> <span class="n">kafkaParams</span><span class="o">,</span> <span class="n">topics</span><span class="o">,</span> <span class="o">...)</span>
<span class="o">}</span>
<span class="c1">//> collection of five *input* DStreams = handled by five receivers/tasks</span>
<span class="k">val</span> <span class="nv">unionDStream</span> <span class="k">=</span> <span class="nv">ssc</span><span class="o">.</span><span class="py">union</span><span class="o">(</span><span class="n">kafkaDStreams</span><span class="o">)</span> <span class="c1">// often unnecessary, just showcasing how to do it</span>
<span class="c1">//> single DStream</span>
<span class="k">val</span> <span class="nv">processingParallelism</span> <span class="k">=</span> <span class="mi">20</span>
<span class="k">val</span> <span class="nv">processingDStream</span> <span class="k">=</span> <span class="nf">unionDStream</span><span class="o">(</span><span class="n">processingParallelism</span><span class="o">)</span>
<span class="c1">//> single DStream but now with 20 partitions</span>
</code></pre></div></div>
<p>In the next section we tie all the pieces together and also cover the actual data processing.</p>
<h2 id="writing-to-kafka">Writing to Kafka</h2>
<p>Writing to Kafka should be done from the <code class="language-plaintext highlighter-rouge">foreachRDD</code> output operation:</p>
<blockquote>
<p>The most generic output operator that applies a function, func, to each RDD generated from the stream. This function should push the data in each RDD to a external system, like saving the RDD to files, or writing it over the network to a database. Note that the function func is executed at the driver, and will usually have RDD actions in it that will force the computation of the streaming RDDs.</p>
</blockquote>
<div class="note">
Note: The remark "the function <tt>func</tt> is executed at the driver" does not mean that, say, a Kafka producer itself
would be run from the driver. Rather, read this remark more as "the function <tt>func</tt> is <em>evaluated</em> at the
driver". The actual behavior will become more clear once you read <em>Design Patterns for using foreachRDD</em>.
</div>
<p>You should read the section
<a href="http://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#output-operations-on-dstreams">Design Patterns for using foreachRDD</a>
in the Spark docs, which explains the recommended patterns as well as common pitfalls when using <code class="language-plaintext highlighter-rouge">foreachRDD</code> to talk to
external systems.</p>
<p>In my case, I decided to follow the recommendation to re-use Kafka producer instances across multiple RDDs/batches via
a pool of producers. I implemented such a pool with <a href="http://commons.apache.org/proper/commons-pool/">Apache Commons Pool</a>,
see <a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/kafka/PooledKafkaProducerAppFactory.scala">PooledKafkaProducerAppFactory</a>.
Factories are helpful in this context because of Spark’s execution and serialization model. The pool itself is provided
to the tasks via a <a href="http://spark.apache.org/docs/latest/programming-guide.html#broadcast-variables">broadcast variable</a>.</p>
<p>The end result looks as follows:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">producerPool</span> <span class="k">=</span> <span class="o">{</span>
<span class="c1">// See the full code on GitHub for details on how the pool is created</span>
<span class="k">val</span> <span class="nv">pool</span> <span class="k">=</span> <span class="nf">createKafkaProducerPool</span><span class="o">(</span><span class="nv">kafkaZkCluster</span><span class="o">.</span><span class="py">kafka</span><span class="o">.</span><span class="py">brokerList</span><span class="o">,</span> <span class="nv">outputTopic</span><span class="o">.</span><span class="py">name</span><span class="o">)</span>
<span class="nv">ssc</span><span class="o">.</span><span class="py">sparkContext</span><span class="o">.</span><span class="py">broadcast</span><span class="o">(</span><span class="n">pool</span><span class="o">)</span>
<span class="o">}</span>
<span class="nv">stream</span><span class="o">.</span><span class="py">map</span> <span class="o">{</span> <span class="o">...</span> <span class="o">}.</span><span class="py">foreachRDD</span><span class="o">(</span><span class="n">rdd</span> <span class="k">=></span> <span class="o">{</span>
<span class="nv">rdd</span><span class="o">.</span><span class="py">foreachPartition</span><span class="o">(</span><span class="n">partitionOfRecords</span> <span class="k">=></span> <span class="o">{</span>
<span class="c1">// Get a producer from the shared pool</span>
<span class="k">val</span> <span class="nv">p</span> <span class="k">=</span> <span class="nv">producerPool</span><span class="o">.</span><span class="py">value</span><span class="o">.</span><span class="py">borrowObject</span><span class="o">()</span>
<span class="nv">partitionOfRecords</span><span class="o">.</span><span class="py">foreach</span> <span class="o">{</span> <span class="k">case</span> <span class="n">tweet</span><span class="k">:</span> <span class="kt">Tweet</span> <span class="o">=></span>
<span class="c1">// Convert pojo back into Avro binary format</span>
<span class="k">val</span> <span class="nv">bytes</span> <span class="k">=</span> <span class="nv">converter</span><span class="o">.</span><span class="py">value</span><span class="o">.</span><span class="py">apply</span><span class="o">(</span><span class="n">tweet</span><span class="o">)</span>
<span class="c1">// Send the bytes to Kafka</span>
<span class="nv">p</span><span class="o">.</span><span class="py">send</span><span class="o">(</span><span class="n">bytes</span><span class="o">)</span>
<span class="o">}</span>
<span class="c1">// Returning the producer to the pool also shuts it down</span>
<span class="nv">producerPool</span><span class="o">.</span><span class="py">value</span><span class="o">.</span><span class="py">returnObject</span><span class="o">(</span><span class="n">p</span><span class="o">)</span>
<span class="o">})</span>
<span class="o">})</span>
</code></pre></div></div>
<p>Keep in mind that Spark Streaming creates many RRDs per minute, each of which contains multiple partitions, so
preferably you shouldn’t create new Kafka producers for each partition, let alone for each Kafka message. The setup
above minimizes the creation of Kafka producer instances, and also minimizes the number of TCP connections that are
being established with the Kafka cluster. You can use this pool setup to precisely control the number of Kafka producer
instances that are being made available to your streaming application (if in doubt, use fewer).</p>
<h2 id="complete-example">Complete example</h2>
<p>The code example below is the gist of my example Spark Streaming application
(<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/spark/KafkaSparkStreamingSpec.scala">see the full code for details and explanations</a>).
Here, I demonstrate how to:</p>
<ul>
<li>Read Avro-encoded data (the <code class="language-plaintext highlighter-rouge">Tweet</code> class) from a Kafka topic in parallel. We use a the optimal read parallelism of
one single-threaded input DStream per Kafka partition.</li>
<li>Deserialize the Avro-encoded data back into pojos, then serializing them back into binary. The serialization is
performed via <a href="https://github.com/twitter/bijection">Twitter Bijection</a>.</li>
<li>Write the results back into a different Kafka topic via a Kafka producer pool.</li>
</ul>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Set up the input DStream to read from Kafka (in parallel)</span>
<span class="k">val</span> <span class="nv">kafkaStream</span> <span class="k">=</span> <span class="o">{</span>
<span class="k">val</span> <span class="nv">sparkStreamingConsumerGroup</span> <span class="k">=</span> <span class="s">"spark-streaming-consumer-group"</span>
<span class="k">val</span> <span class="nv">kafkaParams</span> <span class="k">=</span> <span class="nc">Map</span><span class="o">(</span>
<span class="s">"zookeeper.connect"</span> <span class="o">-></span> <span class="s">"zookeeper1:2181"</span><span class="o">,</span>
<span class="s">"group.id"</span> <span class="o">-></span> <span class="s">"spark-streaming-test"</span><span class="o">,</span>
<span class="s">"zookeeper.connection.timeout.ms"</span> <span class="o">-></span> <span class="s">"1000"</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">inputTopic</span> <span class="k">=</span> <span class="s">"input-topic"</span>
<span class="k">val</span> <span class="nv">numPartitionsOfInputTopic</span> <span class="k">=</span> <span class="mi">5</span>
<span class="k">val</span> <span class="nv">streams</span> <span class="k">=</span> <span class="o">(</span><span class="mi">1</span> <span class="n">to</span> <span class="n">numPartitionsOfInputTopic</span><span class="o">)</span> <span class="n">map</span> <span class="o">{</span> <span class="k">_</span> <span class="k">=></span>
<span class="nv">KafkaUtils</span><span class="o">.</span><span class="py">createStream</span><span class="o">(</span><span class="n">ssc</span><span class="o">,</span> <span class="n">kafkaParams</span><span class="o">,</span> <span class="nc">Map</span><span class="o">(</span><span class="n">inputTopic</span> <span class="o">-></span> <span class="mi">1</span><span class="o">),</span> <span class="nv">StorageLevel</span><span class="o">.</span><span class="py">MEMORY_ONLY_SER</span><span class="o">).</span><span class="py">map</span><span class="o">(</span><span class="nv">_</span><span class="o">.</span><span class="py">_2</span><span class="o">)</span>
<span class="o">}</span>
<span class="k">val</span> <span class="nv">unifiedStream</span> <span class="k">=</span> <span class="nv">ssc</span><span class="o">.</span><span class="py">union</span><span class="o">(</span><span class="n">streams</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">sparkProcessingParallelism</span> <span class="k">=</span> <span class="mi">1</span> <span class="c1">// You'd probably pick a higher value than 1 in production.</span>
<span class="nv">unifiedStream</span><span class="o">.</span><span class="py">repartition</span><span class="o">(</span><span class="n">sparkProcessingParallelism</span><span class="o">)</span>
<span class="o">}</span>
<span class="c1">// We use accumulators to track global "counters" across the tasks of our streaming app</span>
<span class="k">val</span> <span class="nv">numInputMessages</span> <span class="k">=</span> <span class="nv">ssc</span><span class="o">.</span><span class="py">sparkContext</span><span class="o">.</span><span class="py">accumulator</span><span class="o">(</span><span class="mi">0L</span><span class="o">,</span> <span class="s">"Kafka messages consumed"</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">numOutputMessages</span> <span class="k">=</span> <span class="nv">ssc</span><span class="o">.</span><span class="py">sparkContext</span><span class="o">.</span><span class="py">accumulator</span><span class="o">(</span><span class="mi">0L</span><span class="o">,</span> <span class="s">"Kafka messages produced"</span><span class="o">)</span>
<span class="c1">// We use a broadcast variable to share a pool of Kafka producers, which we use to write data from Spark to Kafka.</span>
<span class="k">val</span> <span class="nv">producerPool</span> <span class="k">=</span> <span class="o">{</span>
<span class="k">val</span> <span class="nv">pool</span> <span class="k">=</span> <span class="nf">createKafkaProducerPool</span><span class="o">(</span><span class="nv">kafkaZkCluster</span><span class="o">.</span><span class="py">kafka</span><span class="o">.</span><span class="py">brokerList</span><span class="o">,</span> <span class="nv">outputTopic</span><span class="o">.</span><span class="py">name</span><span class="o">)</span>
<span class="nv">ssc</span><span class="o">.</span><span class="py">sparkContext</span><span class="o">.</span><span class="py">broadcast</span><span class="o">(</span><span class="n">pool</span><span class="o">)</span>
<span class="o">}</span>
<span class="c1">// We also use a broadcast variable for our Avro Injection (Twitter Bijection)</span>
<span class="k">val</span> <span class="nv">converter</span> <span class="k">=</span> <span class="nv">ssc</span><span class="o">.</span><span class="py">sparkContext</span><span class="o">.</span><span class="py">broadcast</span><span class="o">(</span><span class="nv">SpecificAvroCodecs</span><span class="o">.</span><span class="py">toBinary</span><span class="o">[</span><span class="kt">Tweet</span><span class="o">])</span>
<span class="c1">// Define the actual data flow of the streaming job</span>
<span class="nv">kafkaStream</span><span class="o">.</span><span class="py">map</span> <span class="o">{</span> <span class="k">case</span> <span class="n">bytes</span> <span class="k">=></span>
<span class="n">numInputMessages</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="c1">// Convert Avro binary data to pojo</span>
<span class="nv">converter</span><span class="o">.</span><span class="py">value</span><span class="o">.</span><span class="py">invert</span><span class="o">(</span><span class="n">bytes</span><span class="o">)</span> <span class="k">match</span> <span class="o">{</span>
<span class="k">case</span> <span class="nc">Success</span><span class="o">(</span><span class="n">tweet</span><span class="o">)</span> <span class="k">=></span> <span class="n">tweet</span>
<span class="k">case</span> <span class="nc">Failure</span><span class="o">(</span><span class="n">e</span><span class="o">)</span> <span class="k">=></span> <span class="c1">// ignore if the conversion failed</span>
<span class="o">}</span>
<span class="o">}.</span><span class="py">foreachRDD</span><span class="o">(</span><span class="n">rdd</span> <span class="k">=></span> <span class="o">{</span>
<span class="nv">rdd</span><span class="o">.</span><span class="py">foreachPartition</span><span class="o">(</span><span class="n">partitionOfRecords</span> <span class="k">=></span> <span class="o">{</span>
<span class="k">val</span> <span class="nv">p</span> <span class="k">=</span> <span class="nv">producerPool</span><span class="o">.</span><span class="py">value</span><span class="o">.</span><span class="py">borrowObject</span><span class="o">()</span>
<span class="nv">partitionOfRecords</span><span class="o">.</span><span class="py">foreach</span> <span class="o">{</span> <span class="k">case</span> <span class="n">tweet</span><span class="k">:</span> <span class="kt">Tweet</span> <span class="o">=></span>
<span class="c1">// Convert pojo back into Avro binary format</span>
<span class="k">val</span> <span class="nv">bytes</span> <span class="k">=</span> <span class="nv">converter</span><span class="o">.</span><span class="py">value</span><span class="o">.</span><span class="py">apply</span><span class="o">(</span><span class="n">tweet</span><span class="o">)</span>
<span class="c1">// Send the bytes to Kafka</span>
<span class="nv">p</span><span class="o">.</span><span class="py">send</span><span class="o">(</span><span class="n">bytes</span><span class="o">)</span>
<span class="n">numOutputMessages</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="o">}</span>
<span class="nv">producerPool</span><span class="o">.</span><span class="py">value</span><span class="o">.</span><span class="py">returnObject</span><span class="o">(</span><span class="n">p</span><span class="o">)</span>
<span class="o">})</span>
<span class="o">})</span>
<span class="c1">// Run the streaming job</span>
<span class="nv">ssc</span><span class="o">.</span><span class="py">start</span><span class="o">()</span>
<span class="nv">ssc</span><span class="o">.</span><span class="py">awaitTermination</span><span class="o">()</span>
</code></pre></div></div>
<p><em><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/spark/KafkaSparkStreamingSpec.scala">See the full source code for further details and explanations.</a></em></p>
<p>Personally, I really like the conciseness and expressiveness of the Spark Streaming code. As Bobby Evans and Tom Graves
are eluding to in their talk, the Storm equivalent of this code is more verbose and comparatively lower level:
The <a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/KafkaStormSpec.scala">KafkaStormSpec</a>
in <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> wires and runs a Storm topology that performs
the same computations. Well, the spec file itself is only a few lines of code once you exclude the code comments,
which I only keep for didactic reasons; however, keep in mind that in Storm’s Java API you cannot use Scala-like
anonymous functions as I show in the Spark Streaming example above (e.g. the <code class="language-plaintext highlighter-rouge">map</code> and <code class="language-plaintext highlighter-rouge">foreach</code> steps). Instead you
must write “full” classes – bolts in plain Storm, functions/filters in Storm Trident – to achieve the
same functionality, see e.g.
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/bolts/AvroDecoderBolt.scala">AvroDecoderBolt</a>.
This feels a bit similar to, say, having to code against Spark’s own API using Java, where juggling with anonymous
functions is IMHO just as painful.</p>
<p>Lastly, I also liked the <a href="http://spark.apache.org/documentation.html">Spark documentation</a>. It was very easy to get
started, and even some more advanced use is covered (e.g.
<a href="http://spark.apache.org/docs/1.1.0/tuning.html">Tuning Spark</a>). I still had to browse the mailing list and also dive
into the source code, but the general starting experience was ok – only the Kafka integration part was lacking (hence
this blog post). Good job to everyone involved maintaining the docs!</p>
<h1 id="known-issues-in-spark-streaming">Known issues in Spark Streaming</h1>
<div class="note">
Update Jan 20, 2015: Spark 1.2+ includes features such as write ahead logs (WAL) that help to minimize some of the
data loss scenarios for Spark Streaming that are described below. See
<a href="http://databricks.com/blog/2015/01/15/improved-driver-fault-tolerance-and-zero-data-loss-in-spark-streaming.html">Improved Fault-tolerance and Zero Data Loss in Spark Streaming</a>.
</div>
<p>You might have guessed by now that there are indeed a number of unresolved issues in Spark Streaming. I try to
summarize my findings below.</p>
<p>On the one hand there are issues due to some confusion about how to correctly read from and write to Kafka, which you
can follow in mailing list discussions such as
<a href="http://apache-spark-user-list.1001560.n3.nabble.com/Multiple-Kafka-Receivers-and-Union-td14901.html">Multiple Kafka Receivers and Union</a>
and <a href="http://apache-spark-user-list.1001560.n3.nabble.com/How-to-scale-more-consumer-to-Kafka-stream-td13883.html">How to scale more consumer to Kafka stream </a>.</p>
<p>On the other hand there are apparently still some inherent issues in Spark Streaming as well as Spark itself,
notably with regard to data loss in failure scenarios. In other words, issues that you do not want to run into in
production!</p>
<ul>
<li>The current (v1.1) driver in Spark does not recover such raw data that has been received but not processed
(<a href="https://www.mail-archive.com/user@spark.apache.org/msg10572.html">source</a>). Here, your Spark application
may lose data under certain conditions. Tathagata Das points out that driver recovery should be fixed in
Spark v1.2, which will be released around the end of 2014.</li>
<li>The current Kafka “connector” of Spark is based on Kafka’s high-level consumer API. One effect of this is that Spark
Streaming cannot rely on its <code class="language-plaintext highlighter-rouge">KafkaInputDStream</code> to properly replay data from Kafka in case of a downstream data loss
(e.g. Spark machines died).
<ul>
<li>Some people even advocate that the current
<a href="http://markmail.org/message/2lb776ta5sq6lgtw">Kafka connector of Spark should not be used in production</a>
because it is based on the high-level consumer API of Kafka. Instead Spark should use the simple consumer API
(like Storm’s Kafka spout does), which allows you to control offsets and partition assignment deterministically.</li>
</ul>
</li>
<li>The Spark community has been working on filling the previously mentioned gap with e.g. Dibyendu
Bhattacharya’s <a href="https://github.com/dibbhatt/kafka-spark-consumer">kafka-spark-consumer</a>. The latter is a port of
Apache Storm’s <a href="https://github.com/apache/storm/tree/master/external/storm-kafka">Kafka spout</a>, which is based on
Kafka’s so-called simple consumer API, which provides better replaying control in case of downstream failures.</li>
<li>Even given those volunteer efforts, the Spark team would prefer to not special-case data recovery for Kafka, as their
goal is “to provide strong guarantee, exactly-once semantics in all transformations”
(<a href="https://www.mail-archive.com/user@spark.apache.org/msg10572.html">source</a>), which is understandable.
On the flip side it still feels a bit like a wasted opportunity to not leverage Kafka’s built-in replaying
capabilities. Tough call!</li>
<li><a href="https://spark-project.atlassian.net/browse/SPARK-1340">SPARK-1340</a>: In the case of Kafka input DStreams, receivers
are not getting restarted if the worker running the receiver fails. So if a worker dies in production, you will
simply miss the data the receiver(s) was/were responsible to retrieve from Kafka.</li>
<li>See also
<a href="http://spark.apache.org/docs/latest/streaming-programming-guide.html#failure-of-a-worker-node">Failure of a Worker Node</a>
for further discussions on data loss scenarios (“lost input data!”) as well as data duplication scenarios (“wrote
output data twice!”). Applies to Kafka, too.</li>
<li>Spark’s usage of the Kafka consumer parameter
<a href="http://kafka.apache.org/documentation.html#consumerconfigs">auto.offset.reset</a> is different from Kafka’s semantics.
In Kafka, the behavior of setting <code class="language-plaintext highlighter-rouge">auto.offset.reset</code> to “smallest” is that the consumer will automatically reset the
offset to the smallest offset when a) there is no existing offset stored in ZooKeeper or b) there is an existing
offset but it is out of range. Spark however will <em>always</em> remove existing offsets and then start all the way from
zero again. This means whenever you restart your application with <code class="language-plaintext highlighter-rouge">auto.offset.reset = "smallest"</code>, your application
will completely re-process all available Kafka data. Doh!
See <a href="http://apache-spark-user-list.1001560.n3.nabble.com/spark-streaming-and-the-spark-shell-tp3347p3387.html">this discussion</a> and
<a href="http://markmail.org/message/257a5l3oqyftsjxj">that discussion</a>.</li>
<li><a href="https://spark-project.atlassian.net/browse/SPARK-1341">SPARK-1341</a>: Ability to control the data rate in Spark
Streaming. This is relevant in so far that if you are already in trouble because of the other Kafka-relatd issues
above (e.g. the <code class="language-plaintext highlighter-rouge">auto.offset.reset</code> misbehavior), then what may happen is that your streaming application must
or thinks it must re-process a lot of older data. But since there is no built-in rate limitation this may cause your
workers to become overwhelmed and run out of memory.</li>
</ul>
<p>Apart from those failure handling and Kafka-focused issues there are also scaling and stability concerns. Again, please
refer to the
<a href="http://yahoohadoop.tumblr.com/post/98213421641/storm-and-spark-at-yahoo-why-chose-one-over-the-other">Spark and Storm</a>
talk of Bobby and Tom for further details. Both of them have more experience with Spark than I do.</p>
<p>I also came across <a href="https://www.mail-archive.com/user@spark.apache.org/msg11505.html">one comment</a> that there may be
issues with the (awesome!) G1 garbage collector that is available in Java 1.7.0u4+, but I didn’t run into any such issue
so far.</p>
<h1 id="spark-tips-and-tricks">Spark tips and tricks</h1>
<p>I compiled a list of notes while I was implementing the example code. This list is by no means a comprehensive
guide, but it may serve you as a starting point when implementing your own Spark Streaming jobs. It contains
references to the
<a href="http://spark.apache.org/docs/1.1.0/streaming-programming-guide.html">Spark Streaming programming guide</a> as well as
information compiled from the spark-user mailing list.</p>
<h2 id="general">General</h2>
<ul>
<li>When creating your Spark context pay special attention to the configuration that sets the number of cores used by
Spark. You must configure enough cores for running both all the required for <em>receivers</em> (see below) and for the
actual data processing part. In Spark, each receiver is responsible for exactly one input DStream, and each receiver
(and thus each input DStream) occies one core – the only exception is when reading from a file stream
(<a href="http://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#input-dstreams">see docs</a>). So if, for
instance, your job reads from 2 input streams but only has access to 2 cores than the data will be read but no
processing will happen.
<ul>
<li>Note that in a streaming application, you can create multiple input DStreams to receive multiple streams of data
in parallel. I demonstrate such a setup in the example job where we parallelize reading from Kafka.</li>
</ul>
</li>
<li>You can use <a href="http://spark.apache.org/docs/1.1.0/programming-guide.html#broadcast-variables">broadcast variables</a> to
share common, read-only variables across machines (see also the relevant section in the
<a href="http://spark.apache.org/docs/1.1.0/tuning.html#broadcasting-large-variables">Tuning Guide</a>). In the example job I
use broadcast variables to share a) a Kafka producer pool (through which the job writes its output to Kafka) and b)
an injection for encoding/decoding Avro data (from Twitter Bijection).
<a href="http://spark.apache.org/docs/1.1.0/programming-guide.html#passing-functions-to-spark">Passing functions to Spark</a>.</li>
<li>You can use <a href="http://spark.apache.org/docs/1.1.0/programming-guide.html#accumulators">accumulator</a> variables to track
global “counters” across the tasks of your streaming job (think: Hadoop job counters). In the example job I use
accumulators to track how many total messages the job has been consumed from and produced to Kafka, respectively.
If you give your accumulators a name (see link), then they will also be displayed in the Spark UI.</li>
<li>
<p>Do not forget to import the relevant implicits of Spark in general and Spark Streaming in particular:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>// Required to gain access to RDD transformations via implicits.
import org.apache.spark.SparkContext._
// Required when working on `PairDStreams` to gain access to e.g. `DStream.reduceByKey`
// (versus `DStream.transform(rddBatch => rddBatch.reduceByKey()`) via implicits.
//
// See also http://spark.apache.org/docs/1.1.0/programming-guide.html#working-with-key-value-pairs
import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions
</code></pre></div> </div>
</li>
<li>If you’re a fan of <a href="https://github.com/twitter/algebird">Twitter Algebird</a>, then you will like how you can leverage
Count-Min Sketch and friends in Spark. Typically you’ll use operations such as <code class="language-plaintext highlighter-rouge">reduce</code> or <code class="language-plaintext highlighter-rouge">reduceByWindow</code> (cf.
<a href="http://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#transformations-on-dstreams">transformations on DStreams</a>).
The Spark project includes examples for
<a href="https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala">Count-Min Sketch</a>
and
<a href="https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala">HyperLogLog</a>.</li>
<li>If you need to determine the memory consumption of, say, your fancy Algebird data structure – e.g. Count-Min Sketch,
HyperLogLog, or Bloom Filters – as it is being used in your Spark application, then the <code class="language-plaintext highlighter-rouge">SparkContext</code> logs might be
an option for you. See
<a href="http://spark.apache.org/docs/1.1.0/tuning.html#determining-memory-consumption">Determining Memory Consumption</a>.</li>
</ul>
<h2 id="kafka-integration">Kafka integration</h2>
<p>Beyond what I already said in the article above:</p>
<ul>
<li>You may need to tweak the Kafka consumer configuration of Spark Streaming. For example, if you need to read
large messages from Kafka you must increase the <code class="language-plaintext highlighter-rouge">fetch.message.max.bytes</code> consumer setting. You can pass such custom
Kafka parameters to Spark Streaming when calling <code class="language-plaintext highlighter-rouge">KafkaUtils.createStream(...)</code>.</li>
</ul>
<h2 id="testing">Testing</h2>
<ul>
<li>Make sure you stop the <code class="language-plaintext highlighter-rouge">StreamingContext</code> and/or <code class="language-plaintext highlighter-rouge">SparkContext</code> (via <code class="language-plaintext highlighter-rouge">stop()</code>) within a <code class="language-plaintext highlighter-rouge">finally</code> block or your test
framework’s <code class="language-plaintext highlighter-rouge">tearDown method</code>, as Spark does not support two contexts running concurrently in the same program (or
JVM?). (<a href="http://spark.apache.org/docs/1.1.0/programming-guide.html#accumulators">source</a>)</li>
<li>In my experience, when using sbt, you want to configure your build to fork JVMs during testing. At least in the case
of <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> the tests must run several threads in
parallel, e.g. in-memory instances of ZooKeeper, Kafka, Spark. See
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/build.sbt">build.sbt</a> for a starting point.</li>
<li>Also, if you are on Mac OS X, you may want to disable IPv6 in your JVMs to prevent DNS-related timeouts. This issue
is unrelated to Spark. See <a href="https://github.com/miguno/kafka-storm-starter/blob/develop/.sbtopts">.sbtopts</a> for how
to do disable IPv6.</li>
</ul>
<h2 id="performance-tuning">Performance tuning</h2>
<ul>
<li>Make sure you understand the runtime implications of your job if it needs to talk to external systems such as Kafka.
You should read the section <em>Design Patterns for using foreachRDD</em> in the
<a href="http://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#output-operations-on-dstreams">Spark Streaming programming guide</a>.
For instance, my example application uses a pool of Kafka producers to optimize writing from Spark Streaming to Kafka.
Here, “optimizing” means sharing the same (few) producers across tasks, notably to reduce the number of new TCP
connections being established with the Kafka cluster.</li>
<li>Use Kryo for serialization instead of the (slow) default Java serialization (see
<a href="http://spark.apache.org/docs/1.1.0/tuning.html#serialized-rdd-storage">Tuning Spark</a>). My example enables Kryo
and registers e.g. the Avro-generated Java classes with Kryo to speed up serialization. See
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/spark/serialization/KafkaSparkStreamingRegistrator.scala">KafkaSparkStreamingRegistrator</a>.
By the way, the use of Kryo is recommended in Spark for the very same reason it is recommended in Storm.</li>
<li>Configure Spark Streaming jobs to clear persistent RDDs by setting <code class="language-plaintext highlighter-rouge">spark.streaming.unpersist</code> to <code class="language-plaintext highlighter-rouge">true</code>.
This is likely to reduce the RDD memory usage of Spark, potentially improving GC behavior as well.
(<a href="http://spark.apache.org/docs/1.1.0/streaming-programming-guide.html#memory-tuning">source</a>)</li>
<li>Start your P&S tests with storage level <code class="language-plaintext highlighter-rouge">MEMORY_ONLY_SER</code> (here, RDD are stored as serialized Java objects, one byte
array per partition). This is generally more space-efficient than deserialized objects, especially when using a fast
serializer like Kryo, but more CPU-intensive to read. This option is often the best for Spark Streaming jobs.
For local testing you may want to not use the <code class="language-plaintext highlighter-rouge">*_2</code> variants (<code class="language-plaintext highlighter-rouge">2</code> = replication factor).</li>
</ul>
<h1 id="wrapping-up">Wrapping up</h1>
<p>The full Spark Streaming code is available in <a href="https://github.com/miguno/kafka-storm-starter/">kafka-storm-starter</a>.
I’d recommend to begin reading with the
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/spark/KafkaSparkStreamingSpec.scala">KafkaSparkStreamingSpec</a>.
This spec launches in-memory instances of Kafka, ZooKeeper, and Spark, and then runs the example streaming application I
covered in this post.</p>
<p>In summary I enjoyed my initial Spark Streaming experiment. While there are still several problems with Spark/Spark
Streaming that need to be sorted out, I am sure the Spark community will eventually be able to address those. I have
found the Spark community to be positive and willing to help, and I am looking forward to what will be happening over
the next few months.</p>
<p>Given that Spark Streaming still needs some <a href="http://en.wiktionary.org/wiki/tender_loving_care">TLC</a> to reach Storm’s
capabilities in large-scale production settings, would I use it in 24x7 production? Most likely not, with the addendum
“not yet”. So where would I use Spark Streaming in its current state right now? Here are two ideas, and I am sure
there are even more:</p>
<ol>
<li>It seems a good fit to prototype data flows very rapidly. If you run into scalability issues because your data
flows are too large, you can e.g. opt to run Spark Streaming against only a sample or subset of the data.</li>
<li>What about combining Storm and Spark Streaming? For example, you could use Storm to crunch the raw, large-scale
input data down to manageable levels, and then perform follow-up analysis with Spark Streaming, benefitting from the
latter’s out-of-the-box support for many interesting algorithms and computations.
use cases.</li>
</ol>
<p>Thanks to the Spark community for all their great work!</p>
<h1 id="references">References</h1>
<ul>
<li><a href="https://spark.apache.org/docs/latest/streaming-kafka-integration.html">Spark Streaming + Kafka Integration Guide</a></li>
<li><a href="http://www.slideshare.net/spark-project/deep-divewithsparkstreaming-tathagatadassparkmeetup20130617">Deep Dive with Spark Streaming</a>, by Tathagata Das, Jun 2013</li>
<li>Mailing list discussions:
<ul>
<li><a href="https://www.mail-archive.com/dev@spark.incubator.apache.org/msg00531.html">Spark Streaming threading model</a>
– also contains some information on how Spark Streaming pushes input data into blocks</li>
<li><a href="http://apache-spark-user-list.1001560.n3.nabble.com/Low-Level-Kafka-Consumer-for-Spark-td11258.html">Low Level Kafka Consumer for Spark</a>
– lots of information about the current state of Kafka integration in Spark Streaming, known issues, possible
remedies, etc.</li>
<li><a href="http://apache-spark-user-list.1001560.n3.nabble.com/How-are-the-executors-used-in-Spark-Streaming-in-terms-of-receiver-and-driver-program-td9336.html">How are the executors used in Spark Streaming in terms of receiver and driver program? </a>
– machines vs. cores vs. executors vs. receivers vs. DStreams in Spark</li>
</ul>
</li>
</ul>Spark Streaming has been getting some attention lately as a real-time data processing tool, often mentioned alongside Apache Storm. If you ask me, no real-time data processing tool is complete without Kafka integration (smile), hence I added an example Spark Streaming application to kafka-storm-starter that demonstrates how to read from Kafka and write to Kafka, using Avro as the data format and Twitter Bijection for handling the data serialization. In this post I will explain this Spark Streaming example in further detail and also shed some light on the current state of Kafka integration in Spark Streaming. All this with the disclaimer that this happens to be my first experiment with Spark Streaming.Apache Storm 0.9 training deck and tutorial2014-09-15T12:00:00+02:002014-09-15T12:00:00+02:00https://www.michael-noll.com/blog/2014/09/15/apache-storm-training-deck-and-tutorial<p>Today I am happy to share an extensive training deck on <a href="http://storm.incubator.apache.org/">Apache Storm</a> version 0.9,
which covers Storm’s core concepts, operating Storm in production, and developing Storm applications. I also discuss
data serialization with <a href="http://avro.apache.org/">Apache Avro</a> and
<a href="https://github.com/twitter/bijection">Twitter Bijection</a>.</p>
<!-- more -->
<p>The training deck (130 slides) is aimed at developers, operations, and architects.</p>
<p><strong>What the training deck covers</strong></p>
<ol>
<li>Introducing Storm: history, Storm adoption in the industry, why Storm</li>
<li>Storm core concepts: topology, data model, spouts and bolts, groupings, parallelism</li>
<li>Operating Storm: architecture, hardware specs, deploying, monitoring</li>
<li>Developing Storm apps: Hello World, creating a bolt, creating a topology, running a topology, integrating Storm and Kafka, testing, data serialization in Storm, example apps (with <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a>), performance and scalability tuning</li>
<li>Playing with Storm using <a href="https://github.com/miguno/wirbelsturm">Wirbelsturm</a></li>
</ol>
<p>Many thanks to the <a href="https://engineering.twitter.com/">Twitter Engineering team</a> (the creators of Storm) and the Apache
Storm open source community!</p>
<p>See also:</p>
<ul>
<li><a href="/blog/2014/08/18/apache-kafka-training-deck-and-tutorial/">Apache Kafka 0.8 training deck and tutorial</a>,
which I published a month ago</li>
</ul>
<iframe src="//www.slideshare.net/slideshow/embed_code/39087523?rel=0" width="597" height="486" frameborder="0" marginwidth="0" marginheight="0" scrolling="no" style="border:1px solid #CCC; border-width:1px; margin-bottom:5px; max-width: 100%;" allowfullscreen=""> </iframe>
<div style="margin-bottom:5px"> <strong> <a href="https://www.slideshare.net/miguno/apache-storm-09-basic-training-verisign" title="Apache Storm 0.9 basic training - Verisign" target="_blank">Apache Storm 0.9 basic training - Verisign</a> </strong> from <strong><a href="http://www.slideshare.net/miguno" target="_blank">Michael Noll</a></strong> </div>Today I am happy to share an extensive training deck on Apache Storm version 0.9, which covers Storm’s core concepts, operating Storm in production, and developing Storm applications. I also discuss data serialization with Apache Avro and Twitter Bijection.Apache Kafka 0.8 training deck and tutorial2014-08-18T12:00:00+02:002014-08-18T12:00:00+02:00https://www.michael-noll.com/blog/2014/08/18/apache-kafka-training-deck-and-tutorial<p>Today I am happy to share an extensive training deck on <a href="http://kafka.apache.org/">Apache Kafka</a> version 0.8, which
covers Kafka’s core concepts, operating Kafka in production, and developing Kafka applications. I also discuss data
serialization with <a href="http://avro.apache.org/">Apache Avro</a> and <a href="https://github.com/twitter/bijection">Twitter Bijection</a>.</p>
<!-- more -->
<div class="warning">
<strong>Update 2015-08-01:</strong>
Shameless plug! Since publishing this Kafka training deck I joined <a href="http://confluent.io/">Confluent Inc.</a> as their Developer Evangelist. Confluent is the US startup founded in 2014 by the creators of Apache Kafka who developed Kafka while at LinkedIn (see this <a href="http://www.forbes.com/sites/alexkonrad/2015/07/08/confluent-raises-24-million-for-data-streams/">Forbes article about Confluent</a>). Next to building the world's best <a href="http://www.confluent.io/product">stream data platform</a> we are also providing <a href="http://www.confluent.io/training">professional Kafka trainings</a>, which go even deeper as well as beyond my extensive training deck below.
<br />
<br />
I can say with confidence that these are the authoritative and most effective Apache Kafka trainings available on the market. But you don't have to take my word for it -- feel free to <a href="http://www.confluent.io/training">take a look yourself</a> and reach out to us if you are interested. <em>—Michael</em>
</div>
<p>The training deck (120 slides) is aimed at developers, operations, and architects.</p>
<p><strong>What the training deck covers</strong></p>
<ol>
<li>Introducing Kafka: history, Kafka at LinkedIn, Kafka adoption in the industry, why Kafka</li>
<li>Kafka core concepts: topics, partitions, replicas, producers, consumers, brokers</li>
<li>Operating Kafka: architecture, hardware specs, deploying, monitoring, performance and scalability tuning</li>
<li>Developing Kafka apps: writing to Kafka, reading from Kafka, testing, serialization, compression, example apps (with
<a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a>)</li>
<li>Playing with Kafka using <a href="https://github.com/miguno/wirbelsturm">Wirbelsturm</a></li>
</ol>
<p>Many thanks to the <a href="https://engineering.linkedin.com/tags/kafka">LinkedIn Engineering team</a> (the creators of Kafka) and
the Apache Kafka open source community!</p>
<p>See also:</p>
<ul>
<li><a href="/blog/2014/09/15/apache-storm-training-deck-and-tutorial/">Apache Storm 0.9 training deck and tutorial</a>,
which I published a month after this training on Kafka</li>
</ul>
<iframe src="//www.slideshare.net/slideshow/embed_code/38083024?rel=0" width="597" height="486" frameborder="0" marginwidth="0" marginheight="0" scrolling="no" style="border:1px solid #CCC; border-width:1px; margin-bottom:5px; max-width: 100%;" allowfullscreen=""> </iframe>
<div style="margin-bottom:5px"> <strong> <a href="https://www.slideshare.net/miguno/apache-kafka-08-basic-training-verisign" title="Apache Kafka 0.8 basic training - Verisign" target="_blank">Apache Kafka 0.8 basic training - Verisign</a> </strong> from <strong><a href="http://www.slideshare.net/miguno" target="_blank">Michael Noll</a></strong> </div>Today I am happy to share an extensive training deck on Apache Kafka version 0.8, which covers Kafka’s core concepts, operating Kafka in production, and developing Kafka applications. I also discuss data serialization with Apache Avro and Twitter Bijection.Integrating Kafka and Storm: Code Examples and State of the Game2014-05-27T16:51:00+02:002014-05-27T16:51:00+02:00https://www.michael-noll.com/blog/2014/05/27/kafka-storm-integration-example-tutorial<p>The only thing that’s even better than <a href="https://kafka.apache.org/">Apache Kafka</a> and
<a href="http://storm.incubator.apache.org/">Apache Storm</a> is to use the two tools in combination. Unfortunately, their
integration can and is still a pretty challenging task, at least judged by the many discussion threads on the respective
mailing lists. In this post I am introducing <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a>,
which contains many code examples that show you how to integrate Apache Kafka 0.8+ with Apache Storm 0.9+, while using
<a href="http://avro.apache.org/">Apache Avro</a> as the data serialization format. I will also briefly summarize the current
state of their integration on a high level to give you additional context of where the two projects are headed in this
regard.</p>
<!-- more -->
<ul id="markdown-toc">
<li><a href="#state-of-the-integration-game" id="markdown-toc-state-of-the-integration-game">State of the (integration) game</a></li>
<li><a href="#kafka-storm-starter" id="markdown-toc-kafka-storm-starter">kafka-storm-starter</a> <ul>
<li><a href="#overview-and-quick-start" id="markdown-toc-overview-and-quick-start">Overview and quick start</a></li>
<li><a href="#features" id="markdown-toc-features">Features</a></li>
<li><a href="#interested-in-more" id="markdown-toc-interested-in-more">Interested in more?</a></li>
</ul>
</li>
<li><a href="#the-quest-to-get-there" id="markdown-toc-the-quest-to-get-there">The quest to get there</a></li>
<li><a href="#conclusion" id="markdown-toc-conclusion">Conclusion</a></li>
</ul>
<div class="note">
<strong>
kafka-storm-starter is available at <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> on GitHub.
</strong>
</div>
<h1 id="state-of-the-integration-game">State of the (integration) game</h1>
<p>For the lazy reader here’s the TL;DR version of Kafka and Storm integration:</p>
<ul>
<li>You can indeed integrate Kafka 0.8.1.1 (latest stable) and Storm 0.9.1-incubating (latest stable). I mention this
explicitly only to clear up any confusion whatsoever that may have resulted from you reading the mailing lists.</li>
<li>The Kafka/Storm integration is, at this time, still more complicated and error prone than it should be. For this
reason I released the code project <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> (more details
below), which should answer most questions you may have when setting out to connect Storm to Kafka for both reading
and writing data. As such kafka-storm-starter can serve as a bootstrapping template to build your own real-time data
processing pipelines with Kafka and Storm.</li>
<li>In the Storm project we are actively working on closing this integration gap. For instance, we have recently
<a href="https://github.com/apache/incubator-storm/tree/master/external/storm-kafka">merged</a> the
<a href="https://github.com/wurstmeister/storm-kafka-0.8-plus">most popular Kafka spout</a> into the core Storm project.
This Kafka spout will be included in the next version of Storm, 0.9.2-incubating, which is just around the corner.
And the spout is now <a href="https://issues.apache.org/jira/browse/STORM-331">compatible with the latest Kafka 0.8.1.1</a>.
Kudos to <a href="https://twitter.com/ptgoetz">P. Taylor Goetz</a> of HortonWorks for acting as the initial sponsor of the
storm-kafka component! For more information see
<a href="https://github.com/apache/incubator-storm/tree/master/external/storm-kafka">external/storm-kafka</a> in the Storm code
base.</li>
<li>The Kafka project is working on an improved, consolidated consumer API for Kafka 0.9. Take a look at the respective
discussions in the <a href="http://grokbase.com/t/kafka/users/142avhm32j/new-consumer-api-discussion">kafka-user</a> and
<a href="http://grokbase.com/t/kafka/dev/142avhm32j/new-consumer-api-discussion">kafka-dev</a> mailing lists. The
<a href="https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design">Kafka 0.9 Consumer Rewrite Design</a>
document is also worth a read. Moving forward this API initiative should simplify interaction with Kafka in general
and integration with storm-kafka in particular.</li>
</ul>
<h1 id="kafka-storm-starter">kafka-storm-starter</h1>
<h2 id="overview-and-quick-start">Overview and quick start</h2>
<p>A few days ago I released <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> as a means to jumpstart
developers interested in integrating Kafka 0.8 and Storm 0.9. Without further ado let’s take a first quick look.</p>
<p>Before we start we must grab the latest version of the code, which is implemented in Scala 2.10:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ git clone https://github.com/miguno/kafka-storm-starter.git
$ cd kafka-storm-starter
</code></pre></div></div>
<p>We begin the tour by running the test suite:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ ./sbt test
</code></pre></div></div>
<p>Notably this command will run end-to-end tests of Kafka, Storm, and Kafka/Storm integration. See this shortened version
of the test output:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>[...other tests removed...]
[info] KafkaSpec:
[info] Kafka
[info] - should synchronously send and receive a Tweet in Avro format
[info] + Given a ZooKeeper instance
[info] + And a Kafka broker instance
[info] + And some tweets
[info] + And a single-threaded Kafka consumer group
[info] + When I start a synchronous Kafka producer that sends the tweets in Avro binary format
[info] + Then the consumer app should receive the tweets
[info] - should asynchronously send and receive a Tweet in Avro format
[info] + Given a ZooKeeper instance
[info] + And a Kafka broker instance
[info] + And some tweets
[info] + And a single-threaded Kafka consumer group
[info] + When I start an asynchronous Kafka producer that sends the tweets in Avro binary format
[info] + Then the consumer app should receive the tweets
[info] StormSpec:
[info] Storm
[info] - should start a local cluster
[info] + Given no cluster
[info] + When I start a LocalCluster instance
[info] + Then the local cluster should start properly
[info] - should run a basic topology
[info] + Given a local cluster
[info] + And a wordcount topology
[info] + And the input words alice, bob, joe, alice
[info] + When I submit the topology
[info] + Then the topology should properly count the words
[info] KafkaStormSpec:
[info] Feature: AvroDecoderBolt[T]
[info] Scenario: User creates a Storm topology that uses AvroDecoderBolt
[info] Given a ZooKeeper instance
[info] And a Kafka broker instance
[info] And a Storm topology that uses AvroDecoderBolt and that reads tweets from topic testing-input and writes them as-is to topic testing-output
[info] And some tweets
[info] And a synchronous Kafka producer app that writes to the topic testing-input
[info] And a single-threaded Kafka consumer app that reads from topic testing-output
[info] And a Storm topology configuration that registers an Avro Kryo decorator for Tweet
[info] When I run the Storm topology
[info] And I use the Kafka producer app to Avro-encode the tweets and sent them to Kafka
[info] Then the Kafka consumer app should receive the decoded, original tweets from the Storm topology
[info] Feature: AvroScheme[T] for Kafka spout
[info] Scenario: User creates a Storm topology that uses AvroScheme in Kafka spout
[info] Given a ZooKeeper instance
[info] And a Kafka broker instance
[info] And a Storm topology that uses AvroScheme and that reads tweets from topic testing-input and writes them as-is to topic testing-output
[info] And some tweets
[info] And a synchronous Kafka producer app that writes to the topic testing-input
[info] And a single-threaded Kafka consumer app that reads from topic testing-output
[info] And a Storm topology configuration that registers an Avro Kryo decorator for Tweet
[info] When I run the Storm topology
[info] And I use the Kafka producer app to Avro-encode the tweets and sent them to Kafka
[info] Then the Kafka consumer app should receive the decoded, original tweets from the Storm topology
[info] Run completed in 21 seconds, 852 milliseconds.
[info] Total number of tests run: 25
[info] Suites: completed 8, aborted 0
[info] Tests: succeeded 25, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[success] Total time: 22 s, completed May 23, 2014 12:31:09 PM
</code></pre></div></div>
<p>We finish the tour by launching the
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/KafkaStormDemo.scala">KafkaStormDemo</a>
application:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ ./sbt run
</code></pre></div></div>
<p>This demo starts in-memory instances of ZooKeeper, Kafka, and Storm. It then runs a demo Storm topology that connects
to and reads from the Kafka instance.</p>
<p>You will see output similar to the following (some parts removed to improve readability):</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>7031 [Thread-19] INFO backtype.storm.daemon.worker - Worker 3f7f1a51-5c9e-43a5-b431-e39a7272215e for storm kafka-storm-starter-1-1400839826 on daa60807-d440-4b45-94fc-8dd7798453d2:1027 has finished loading
7033 [Thread-29-kafka-spout] INFO storm.kafka.DynamicBrokersReader - Read partition info from zookeeper: GlobalPartitionInformation{partitionMap={0=127.0.0.1:9092}}
7050 [Thread-29-kafka-spout] INFO backtype.storm.daemon.executor - Opened spout kafka-spout:(1)
7051 [Thread-29-kafka-spout] INFO backtype.storm.daemon.executor - Activating spout kafka-spout:(1)
7051 [Thread-29-kafka-spout] INFO storm.kafka.ZkCoordinator - Refreshing partition manager connections
7065 [Thread-29-kafka-spout] INFO storm.kafka.DynamicBrokersReader - Read partition info from zookeeper: GlobalPartitionInformation{partitionMap={0=127.0.0.1:9092}}
7066 [Thread-29-kafka-spout] INFO storm.kafka.ZkCoordinator - Deleted partition managers: []
7066 [Thread-29-kafka-spout] INFO storm.kafka.ZkCoordinator - New partition managers: [Partition{host=127.0.0.1:9092, partition=0}]
7083 [Thread-29-kafka-spout] INFO storm.kafka.PartitionManager - Read partition information from: /kafka-spout/kafka-storm-starter/partition_0 --> null
7100 [Thread-29-kafka-spout] INFO storm.kafka.PartitionManager - No partition information found, using configuration to determine offset
7105 [Thread-29-kafka-spout] INFO storm.kafka.PartitionManager - Starting Kafka 127.0.0.1:0 from offset 18
7106 [Thread-29-kafka-spout] INFO storm.kafka.ZkCoordinator - Finished refreshing
7126 [Thread-29-kafka-spout] INFO storm.kafka.PartitionManager - Committing offset for Partition{host=127.0.0.1:9092, partition=0}
7126 [Thread-29-kafka-spout] INFO storm.kafka.PartitionManager - Committed offset 18 for Partition{host=127.0.0.1:9092, partition=0} for topology: 47e82e34-fb36-427e-bde6-8cd971db2527
9128 [Thread-29-kafka-spout] INFO storm.kafka.PartitionManager - Committing offset for Partition{host=127.0.0.1:9092, partition=0}
9129 [Thread-29-kafka-spout] INFO storm.kafka.PartitionManager - Committed offset 18 for Partition{host=127.0.0.1:9092, partition=0} for topology: 47e82e34-fb36-427e-bde6-8cd971db2527
</code></pre></div></div>
<p>At this point Storm is connected to Kafka (more precisely: to the <code class="language-plaintext highlighter-rouge">testing</code> topic in Kafka). The last few lines from
above – “Committing offset …” — will be repeated again and again, because a) this demo Storm topology only reads
from the Kafka topic but it does nothing to the data that was read and b) because we are not sending any data to the
Kafka topic.</p>
<div class="note">
<strong>Note:</strong> This example will actually run <em>two</em> in-memory instances of ZooKeeper: the first (listening at <tt>127.0.0.1:2181/tcp</tt>) is used by the Kafka instance, the second (listening at <tt>127.0.0.1:2000/tcp</tt>) is automatically started and used by the in-memory Storm cluster. This is because, when running in local aka in-memory mode, Storm does not allow you to reconfigure or disable its own ZooKeeper instance.
</div>
<p><strong>To stop the demo application you must kill or <code class="language-plaintext highlighter-rouge">Ctrl-C</code> the process in the terminal.</strong></p>
<p>You can use
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/KafkaStormDemo.scala">KafkaStormDemo</a>
as a starting point to create your own, “real” Storm topologies that read from a “real” Kafka, Storm, and ZooKeeper
infrastructure. An easy way to get started with such an infrastructure is by deploying Kafka, Storm, and ZooKeeper via
a tool such as <a href="https://github.com/miguno/wirbelsturm">Wirbelsturm</a>.</p>
<h2 id="features">Features</h2>
<p>I showcase the following features in kafka-storm-starter. Note that I focus on showcasing, and not necessarily on
“production ready”.</p>
<ul>
<li>How to integrate Kafka and Storm.</li>
<li>How to use <a href="http://avro.apache.org/">Avro</a> with Kafka and Storm for serializing and deserializing the data payload.
For this I leverage <a href="https://github.com/twitter/bijection">Twitter Bijection</a> and
<a href="https://github.com/twitter/chill/">Twitter Chill</a>.</li>
<li>Kafka standalone code examples
<ul>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/kafka/KafkaProducerApp.scala">KafkaProducerApp</a>:
A simple Kafka producer app for writing Avro-encoded data into Kafka.
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/KafkaSpec.scala">KafkaSpec</a>
puts this producer to use and shows how to use Twitter Bijection to Avro-encode the messages being sent to Kafka.</li>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/kafka/KafkaConsumerApp.scala">KafkaConsumerApp</a>:
A simple Kafka consumer app for reading Avro-encoded data from Kafka.
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/KafkaSpec.scala">KafkaSpec</a>
puts this consumer to use and shows how to use Twitter Bijection to Avro-decode the messages being read from
Kafka.</li>
</ul>
</li>
<li>Storm standalone code examples
<ul>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/AvroDecoderBolt.scala">AvroDecoderBolt[T]</a>:
An <code class="language-plaintext highlighter-rouge">AvroDecoderBolt[T <: org.apache.avro.specific.SpecificRecordBase]</code> that can be parameterized with the type of
the Avro record <code class="language-plaintext highlighter-rouge">T</code> it will deserialize its data to (i.e. no need to write another decoder bolt just because the
bolt needs to handle a different Avro schema).</li>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/AvroScheme.scala">AvroScheme[T]</a>:
An <code class="language-plaintext highlighter-rouge">AvroScheme[T <: org.apache.avro.specific.SpecificRecordBase]</code> scheme, i.e. a custom
<code class="language-plaintext highlighter-rouge">backtype.storm.spout.Scheme</code> to auto-deserialize a spout’s incoming data. The scheme can be parameterized with
the type of the Avro record <code class="language-plaintext highlighter-rouge">T</code> it will deserializes its data to (i.e. no need to write another scheme just
because the scheme needs to handle a different Avro schema).
<ul>
<li>You can opt to configure a spout (such as the Kafka spout) with <code class="language-plaintext highlighter-rouge">AvroScheme</code> if you want to perform the Avro
decoding step directly in the spout instead of placing an <code class="language-plaintext highlighter-rouge">AvroDecoderBolt</code> after the Kafka spout. You may
want to profile your topology which of the two approaches works best for your use case.</li>
</ul>
</li>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/TweetAvroKryoDecorator.scala">TweetAvroKryoDecorator</a>:
A custom <code class="language-plaintext highlighter-rouge">backtype.storm.serialization.IKryoDecorator</code>, i.e. a custom
<a href="http://storm.incubator.apache.org/documentation/Serialization.html">Kryo serializer for Storm</a>.
<ul>
<li>Unfortunately we have not figured out a way to implement a parameterized <code class="language-plaintext highlighter-rouge">AvroKryoDecorator[T]</code> variant yet.
(A “straight-forward” approach we tried – similar to the other parameterized components – compiled fine but
failed at runtime when running the tests). Code contributions are welcome!</li>
</ul>
</li>
</ul>
</li>
<li>Kafka and Storm integration
<ul>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/AvroKafkaSinkBolt.scala">AvroKafkaSinkBolt[T]</a>:
An <code class="language-plaintext highlighter-rouge">AvroKafkaSinkBolt[T <: org.apache.avro.specific.SpecificRecordBase]</code> that can be parameterized with the type
of the Avro record <code class="language-plaintext highlighter-rouge">T</code> it will serialize its data to before sending the encoded data to Kafka (i.e. no
need to write another Kafka sink bolt just because the bolt needs to handle a different Avro schema).</li>
<li>Storm topologies that read Avro-encoded data from Kafka:
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/main/scala/com/miguno/kafkastorm/storm/KafkaStormDemo.scala">KafkaStormDemo</a> and
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/KafkaStormSpec.scala">KafkaStormSpec</a></li>
<li>A Storm topology that writes Avro-encoded data to Kafka:
<a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/KafkaStormSpec.scala">KafkaStormSpec</a></li>
</ul>
</li>
<li>Unit testing
<ul>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/storm/AvroDecoderBoltSpec.scala">AvroDecoderBoltSpec</a></li>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/storm/AvroSchemeSpec.scala">AvroSchemeSpec</a></li>
<li>And more under <a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm">src/test/scala</a></li>
</ul>
</li>
<li>Integration testing
<ul>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/KafkaSpec.scala">KafkaSpec</a>:
Tests for Kafka, which launch and run against in-memory instances of Kafka and ZooKeeper.</li>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/StormSpec.scala">StormSpec</a>:
Tests for Storm, which launch and run against in-memory instances of Storm and ZooKeeper.</li>
<li><a href="https://github.com/miguno/kafka-storm-starter/blob/develop/src/test/scala/com/miguno/kafkastorm/integration/KafkaStormSpec.scala">KafkaStormSpec</a>:
Tests for integrating Storm and Kafka, which launch and run against in-memory instances of Kafka, Storm, and
ZooKeeper.</li>
</ul>
</li>
</ul>
<h2 id="interested-in-more">Interested in more?</h2>
<p>All the gory details are available at <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a>. Apart from
the code and build script (sbt) I provide information about how to create Cobertura code coverage reports, to package
the code, to create Java “sources” and “javadoc” jars, to generate API docs, to integrate with
<a href="http://jenkins-ci.org/">Jenkins CI</a> and <a href="http://www.jetbrains.com/teamcity/">TeamCity</a> build servers, and to set up
kafka-storm-starter as a project in IntelliJ IDEA and Eclipse.</p>
<p>Moving forward my plan is to keep kafka-storm-starter up to date with the latest versions of Kafka and Storm. The
next version of Storm, 0.9.2, will already simplify the current setup quite a lot. Of course I welcome any code, docs,
or similar <a href="https://github.com/miguno/kafka-storm-starter#Contributing">contributions you may have</a>.</p>
<h1 id="the-quest-to-get-there">The quest to get there</h1>
<p>Just for the historical record here are some of the gotchas that are addressed by kafka-storm-starter, i.e. problems
you do not need to solve yourself anymore:</p>
<ul>
<li>Figuring out which Kafka spout in Storm 0.9 works with the latest Kafka 0.8 version. A lot of people tried in vain to
use a Kafka spout built for Kafka 0.7 to read from Kafka 0.8. Others didn’t know how to use the available Kafka 0.8
spouts in their code, and so on. In the case of kafka-storm-starter I opted to go with the spout created by
<a href="https://github.com/wurstmeister/storm-kafka-0.8-plus">wurstmeister</a>, primarily because this spout will soon by the
“official” Kafka spout maintained by the Storm project. Unfortunately the latest version of the spout was/is not
available in a public Maven repository, so I had take care of that, too, until Storm 0.9.2 will provide the official
version.
<ul>
<li>Alternatively you can also try <a href="https://github.com/HolmesNL/kafka-spout">Kafka spout of HolmesNL</a>, developed by
Mattijs Ugen. I don’t want to talk about the differences to the wurstmeister spout in detail, but essentially
the wurstmeister spout uses the
<a href="https://kafka.apache.org/documentation.html#simpleconsumerapi">Simple Consumer API</a> of Kafka 0.8 whereas the
Mattijs’ spout uses the
<a href="https://kafka.apache.org/documentation.html#highlevelconsumerapi">High Level Consumer API</a>.</li>
</ul>
</li>
<li>Resolving version conflicts between the various software packages. For instance, Storm 0.9.1 has a transitive
dependency on Kryo 2.17 because Storm depends on an old version of <a href="https://github.com/sritchie/carbonite">Carbonite</a>.
This causes problems when trying to use Twitter Bijection or Twitter Chill, because those require a newer version of
Kryo. (Apart from that Kryo 2.21 also fixes data corruption issues, so you do want the newer version.) To address
this issue I filed <a href="https://issues.apache.org/jira/browse/STORM-263">STORM-263</a>, which is included in upcoming
Storm 0.9.2. Thanks to <a href="https://twitter.com/sritchie">Sam Ritchie</a>, the maintainer of Carbonite, and everyone else
involved to get the patch included.
Another example is that you must exclude <code class="language-plaintext highlighter-rouge">javax.jms:jms</code> (and a few others) when including Kafka into your build
dependencies. Or how to handle Netflix (now: Apache) Curator conflicts.</li>
<li>Understanding the various conflicting ZooKeeper versions, and picking a version to go with. Right now Storm and Kafka
still prefer very old 3.3.x versions of ZooKeeper, whereas in practice many people run 3.4.x in their infrastructure
(e.g. because ZooKeeper 3.4.x is already deployed alongside other infrastructure pieces such as Hadoop clusters
when using commercial Hadoop distributions).</li>
<li>How to write unit tests for Storm topologies. A lot of people seem to find references to
<a href="https://github.com/xumingming/storm-lib/blob/master/src/jvm/storm/TestingApiDemo.java">TestingApiDemo.java</a> while
searching the Internet but struggle with extracting these examples out of the Storm code base and merging them into
their own project.</li>
<li>How to write Storm topologies in a way that you can parameterize its components (bolts etc.) with the Avro record type
<code class="language-plaintext highlighter-rouge">T</code>, so that you don’t need to write a new bolt only because your Avro schema changes. The goal of this code is to
show how you can improve the developer/user experience by providing ready-to-use functionality, in this case with
regards to (Avro) serialization/deserialization. To tackle this you must understand
<a href="https://storm.incubator.apache.org/documentation/Serialization.html">Storm’s serialization system</a> as well
as its run-time behavior.
<ul>
<li>While doing that I discovered a (known) Scala bug when I tried to use <code class="language-plaintext highlighter-rouge">TypeTag</code> instead of deprecated <code class="language-plaintext highlighter-rouge">Manifest</code>
to implement e.g. <code class="language-plaintext highlighter-rouge">AvroDecoderBolt[T]</code>, see <a href="https://issues.scala-lang.org/browse/SI-5919">SI-5919</a>. This
bug is still not fixed in the latest Scala 2.11.1, by the way.</li>
</ul>
</li>
<li>How to write end-to-end Kafka->Storm->Kafka tests.</li>
<li>And so on…</li>
</ul>
<h1 id="conclusion">Conclusion</h1>
<p>I hope you find <a href="https://github.com/miguno/kafka-storm-starter">kafka-storm-starter</a> useful to bootstrap your own
Kafka/Storm application. In the Storm community we are actively working on improving and simplifying the Kafka/Storm
integration, so please stay tuned and, above all, thanks for your patience. The upcoming 0.9.2 version of Storm is
already a first step in the right direction by bundling a Kafka spout that works with the latest stable version of
Kafka (0.8.1.1 at the time of this writing).</p>
<p>Now where to go once you have your Kafka and Storm code ready? At this point you can then use a tool such as
<a href="https://github.com/miguno/wirbelsturm">Wirbelsturm</a> and its associated Puppet modules to deploy production Kafka and
Storm clusters and run your own real-time data processing pipelines at scale.</p>The only thing that’s even better than Apache Kafka and Apache Storm is to use the two tools in combination. Unfortunately, their integration can and is still a pretty challenging task, at least judged by the many discussion threads on the respective mailing lists. In this post I am introducing kafka-storm-starter, which contains many code examples that show you how to integrate Apache Kafka 0.8+ with Apache Storm 0.9+, while using Apache Avro as the data serialization format. I will also briefly summarize the current state of their integration on a high level to give you additional context of where the two projects are headed in this regard.Wirbelsturm: 1-Click Deployments of Storm and Kafka clusters with Vagrant and Puppet2014-03-17T17:58:00+01:002014-03-17T17:58:00+01:00https://www.michael-noll.com/blog/2014/03/17/wirbelsturm-one-click-deploy-storm-kafka-clusters-with-vagrant-puppet<p>I am happy to announce the first public release of <a href="https://github.com/miguno/wirbelsturm">Wirbelsturm</a>, a Vagrant and
Puppet based tool to perform 1-click local and remote deployments, with a focus on big data related infrastructure.
Wirbelsturm’s goal is to make tasks such as “I want to deploy a multi-node Storm cluster” <em>simple</em>, <em>easy</em>, and <em>fun</em>.
In this post I will introduce you to Wirbelsturm, talk a bit about its history, and show you how to launch a multi-node
Storm (or Kafka or …) cluster faster than you can brew an espresso.</p>
<!-- more -->
<ul id="markdown-toc">
<li><a href="#wirbelsturm-quick-start" id="markdown-toc-wirbelsturm-quick-start">Wirbelsturm quick start</a></li>
<li><a href="#motivation" id="markdown-toc-motivation">Motivation</a></li>
<li><a href="#current-wirbelsturm-features" id="markdown-toc-current-wirbelsturm-features">Current Wirbelsturm features</a></li>
<li><a href="#is-wirbelsturm-for-me" id="markdown-toc-is-wirbelsturm-for-me">Is Wirbelsturm for me?</a></li>
<li><a href="#wirbelsturm-in-detail" id="markdown-toc-wirbelsturm-in-detail">Wirbelsturm in detail</a></li>
<li><a href="#the-long-road-of-getting-there" id="markdown-toc-the-long-road-of-getting-there">The long road of getting there</a></li>
<li><a href="#lessons-learned-mistakes-made-along-the-way" id="markdown-toc-lessons-learned-mistakes-made-along-the-way">Lessons learned: mistakes made along the way</a></li>
<li><a href="#summary" id="markdown-toc-summary">Summary</a></li>
<li><a href="#related-work" id="markdown-toc-related-work">Related work</a></li>
</ul>
<div class="note">
<strong>
Wirbelsturm is available at <a href="https://github.com/miguno/wirbelsturm">wirbelsturm</a> on GitHub.
</strong>
</div>
<p><strong>Update May 27, 2014:</strong> If you want to build real-time data processing pipelines based on Kafka and Storm, you may be
interested in <a href="/blog/2014/05/27/kafka-storm-integration-example-tutorial/">kafka-storm-starter</a>. It contains code
examples that show to integrate Apache Kafka 0.8+ with Apache Storm 0.9+, while using Apache Avro as the data
serialization format.</p>
<h1 id="wirbelsturm-quick-start">Wirbelsturm quick start</h1>
<p>This section is an appetizer of what you can do with Wirbelsturm. Do not worry if something is not immediately obvious
to you – the <a href="https://github.com/miguno/wirbelsturm">Wirbelsturm documentation</a> describes everything in full detail.</p>
<p>Assuming you are using a reasonably powerful computer and have already installed <a href="http://www.vagrantup.com/">Vagrant</a>
(1.4.x – 1.5.x is not supported yet) and <a href="https://www.virtualbox.org/">VirtualBox</a> you can launch a multi-node
<a href="http://storm.incubator.apache.org/">Apache Storm</a> cluster on your local machine with the following commands.</p>
<div class="language-bash highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="nv">$ </span>git clone https://github.com/miguno/wirbelsturm.git
<span class="nv">$ </span><span class="nb">cd </span>wirbelsturm
<span class="nv">$ </span>./bootstrap <span class="c"># <<< May take a while depending on how fast your Internet connection is.</span>
<span class="nv">$ </span>vagrant up <span class="c"># <<< ...and this step also depends on how powerful your computer is.</span>
</code></pre></div></div>
<p>Done – you now have a fully functioning Storm cluster up and running on your computer! The deployment should have
taken you significantly less time and effort than
<a href="/tutorials/running-multi-node-storm-cluster/">going through long blog posts</a> or
<a href="http://storm.incubator.apache.org/documentation/Documentation.html">working through the official documentation</a>. On
top of that, you can now re-deploy your setup everywhere and every time you need it, thanks to automation.</p>
<div class="note">
Note: Running a small, local Storm cluster is just the default example. You can do much more with Wirbelsturm than this.
</div>
<p>Let’s take a look at which virtual machines back this cluster behind the scenes:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ vagrant status
Current machine states:
zookeeper1 running (virtualbox)
nimbus1 running (virtualbox)
supervisor1 running (virtualbox)
supervisor2 running (virtualbox)
</code></pre></div></div>
<p>Storm also ships with a web UI that shows you the cluster’s state, e.g. how many nodes it has, whether any processing
jobs (topologies) are being executed, etc. Wait 20-30 seconds after the deployment is done and then open the Storm UI
at <a href="http://localhost:28080/">http://localhost:28080/</a>.</p>
<p><img src="https://www.michael-noll.com/assets/uploads/wirbelsturm-storm-ui-screenshot.png" alt="Storm UI showing two slaves nodes" /></p>
<div class="caption">
Figure 1: The default example of Wirbelsturm deploys a multi-node Storm cluster. In this screenshot of the Storm UI you can see the two slave nodes -- named <em>supervisor1</em> and <em>supervisor2</em> -- running Storm's Supervisor daemons. The third machine acts as the Storm master node and runs the Nimbus daemon and this Storm UI. The fourth machine runs ZooKeeper.
</div>
<p>What’s more, Wirbelsturm also allows you to use <a href="http://www.ansible.com/">Ansible</a> to interact with the deployed
machines via its <a href="https://github.com/miguno/wirbelsturm/blob/master/ansible">ansible</a> wrapper script:</p>
<div class="language-bash highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="nv">$ </span>./ansible all <span class="nt">-m</span> ping
zookeeper1 | success <span class="o">>></span> <span class="o">{</span>
<span class="s2">"changed"</span>: <span class="nb">false</span>,
<span class="s2">"ping"</span>: <span class="s2">"pong"</span>
<span class="o">}</span>
supervisor1 | success <span class="o">>></span> <span class="o">{</span>
<span class="s2">"changed"</span>: <span class="nb">false</span>,
<span class="s2">"ping"</span>: <span class="s2">"pong"</span>
<span class="o">}</span>
nimbus1 | success <span class="o">>></span> <span class="o">{</span>
<span class="s2">"changed"</span>: <span class="nb">false</span>,
<span class="s2">"ping"</span>: <span class="s2">"pong"</span>
<span class="o">}</span>
supervisor2 | success <span class="o">>></span> <span class="o">{</span>
<span class="s2">"changed"</span>: <span class="nb">false</span>,
<span class="s2">"ping"</span>: <span class="s2">"pong"</span>
<span class="o">}</span>
</code></pre></div></div>
<p>Want to run more Storm slaves? As long as your computer has enough horsepower you only need to change a single number
in <code class="language-plaintext highlighter-rouge">wirbelsturm.yaml</code>:</p>
<div class="language-yaml highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1"># wirbelsturm.yaml</span>
<span class="na">nodes</span><span class="pi">:</span>
<span class="s">...</span>
<span class="s">storm_slave</span><span class="err">:</span>
<span class="na">count</span><span class="pi">:</span> <span class="m">2</span> <span class="c1"># <<< changing 2 to 4 is all it takes</span>
<span class="err"> </span><span class="s">...</span>
</code></pre></div></div>
<p>Then run <code class="language-plaintext highlighter-rouge">vagrant up</code> again and shortly after <code class="language-plaintext highlighter-rouge">supervisor3</code> and <code class="language-plaintext highlighter-rouge">supervisor4</code> will be up and running.</p>
<p>Want to run an <a href="http://kafka.apache.org/">Apache Kafka</a> broker? Just uncomment the <code class="language-plaintext highlighter-rouge">kafka_broker</code> section in your
<code class="language-plaintext highlighter-rouge">wirbelsturm.yaml</code> that it looks similar to the following example snippet (only remove the leading <code class="language-plaintext highlighter-rouge">#</code> characters, do not remove any whitespace):</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># wirbelsturm.yaml
nodes:
...
# Deploys Kafka brokers.
kafka_broker:
count: 1
hostname_prefix: kafka
ip_range_start: 10.0.0.20
node_role: kafka_broker
providers:
virtualbox:
memory: 1536
aws:
instance_type: t1.micro
ami: ami-86cdb3ef
security_groups:
- wirbelsturm
...
</code></pre></div></div>
<p>Then run <code class="language-plaintext highlighter-rouge">vagrant up kafka1</code>. Now you have Kafka running alongside Storm.</p>
<p>Once you have finished playing around, you can stop all the machines in the cluster again by executing
<code class="language-plaintext highlighter-rouge">vagrant destroy</code>.</p>
<h1 id="motivation">Motivation</h1>
<p>Let me use an analogy to explain the motivation to build Wirbelsturm. While I assume every last one of us wants to work
somehow like this…</p>
<p><img src="https://www.michael-noll.com/assets/uploads/neo-cool.png" alt="The dream." /></p>
<p>…most of our actual time is rather spent on doing something like that:</p>
<p><img src="https://www.michael-noll.com/assets/uploads/scotty-uncool.png" alt="The reality." /></p>
<p>Without any automated deployment tools the task of setting up cluster environments with (say) Storm or Kafka is simply a
very time-consuming, complicated, and – let’s face it – mind-numbingly boring experience. So the motivation for
Wirbelsturm was really simple: first, minimize frustration, and second, help others.</p>
<p>While these were the primary reasons there were also secondary aspects: Wirbelsturm should integrate nicely with
existing deployment infrastructures and the associated skills of Operations teams – that’s why it is so heavily based
on Puppet, though e.g. Chef and Ansible would have been good candidates, too. Also, it should allow you to perform
local deployments (say, your dev laptop) as well as remote deployments (larger-scale environments, production, etc.) –
that’s why Vagrant was added to the picture. You should also be able to easily transition from a Wirbelsturm/Vagrant
backed setup to a “real” production setup without having to re-architect your deployment, switch tools, etc.</p>
<p>As such Wirbelsturm is one of the tools that help to make the process of going from “Hey, I have this cool idea” to
“It’s live in production!” as simple, easy, and fun as possible. A developer should be free to completely screw up
“his” test environment; two developers in the same team should always have the same copy of an environment; the
integration environment of that team should look and feel the same way, too; and for sure that should apply to
the production environment as well.</p>
<p>I think at this point the motivation should be pretty clear, and in the section <em>Is Wirbelsturm for me?</em> I list further
examples on what you can do with Wirbelsturm.</p>
<h1 id="current-wirbelsturm-features">Current Wirbelsturm features</h1>
<p>In its first public release Wirbelsturm supports the following high-level features:</p>
<ul>
<li><strong>Launching machines:</strong> Wirbelsturm uses Vagrant to launch the machines that make up your infrastructure
as VMs running locally in VirtualBox (default) or remotely in Amazon AWS/EC2 (OpenStack support is in the works).</li>
<li><strong>Provisioning machines:</strong> Machines are provisioned via Puppet.
<ul>
<li>Wirbelsturm uses a master-less Puppet setup, i.e. provisioning is ultimately performed through <code class="language-plaintext highlighter-rouge">puppet apply</code>.</li>
<li>Puppet modules are managed via <a href="https://github.com/rodjek/librarian-puppet">librarian-puppet</a>.</li>
</ul>
</li>
<li><strong>(Some) batteries included:</strong> We maintain a number of standard Puppet modules that work well with Wirbelsturm, some
of which are included in the default configuration of Wirbelsturm. However you can use any Puppet module with
Wirbelsturm, of course. See <a href="#supported-puppet-modules">Supported Puppet modules</a> for more information.</li>
<li><strong>Ansible support:</strong> The <a href="http://www.ansible.com/">Ansible</a> aficionados amongst us can use Ansible to interact with
machines once deployed through Wirbelsturm and Puppet.</li>
<li><strong>Host operating system support:</strong> Wirbelsturm has been tested with Mac OS X 10.8+ and RHEL/CentOS 6 as host machines.
Debian/Ubuntu should work, too.</li>
<li><strong>Guest operating system support:</strong> The target OS version for deployed machines is RHEL/CentOS 6 (64-bit). Amazon
Linux is supported, too.
<ul>
<li>For local deployments (via VirtualBox) and AWS deployments Wirbelsturm uses a
<a href="http://puppet-vagrant-boxes.puppetlabs.com/">CentOS 6 box created by PuppetLabs</a>.</li>
<li>Switching to RHEL 6 only requires specifying a different <a href="http://docs.vagrantup.com/v2/boxes.html">Vagrant box</a>
in <a href="bootstrap">bootstrap</a> (for VirtualBox) or a different AMI image in <code class="language-plaintext highlighter-rouge">wirbelsturm.yaml</code> (for Amazon
AWS).</li>
</ul>
</li>
<li><strong>When using tools other than Vagrant to launch machines:</strong> Wirbelsturm-compatible Puppet modules are standard Puppet
modules, so of course they can be used standalone, too. This way you can deploy against bare metal machines even if
you are not able to or do not want to run Wirbelsturm and/or Vagrant directly.</li>
</ul>
<h1 id="is-wirbelsturm-for-me">Is Wirbelsturm for me?</h1>
<p>Here are some ideas for what you can do with Wirbelsturm:</p>
<ul>
<li>Evaluate new technologies such as Kafka and Storm in a temporary environment that you can set up and tear
down at will. Without having to spend hours and stay late figuring out how to install those tools.
Then tell your boss how hard you worked for it.</li>
<li>Provide your teams with a consistent look and feel of infrastructure environments from initial prototyping
to development & testing and all the way to production. Banish “But it does work fine on <em>my</em> machine!” remarks
from your daily standups. Well, hopefully.</li>
<li>Save money if (at least some of) these environments run locally instead of in an IAAS cloud or on bare-metal
machines that you would need to purchase first. Make Finance happy for the first time.</li>
<li>Create production-like environments for training classes. Use them to get new hires up to speed. Or unleash a
<a href="http://techblog.netflix.com/2012/07/chaos-monkey-released-into-wild.html">Chaos Monkey</a> and check how well your
applications, DevOps tools, or technical staff can handle the mess. Bring coke and popcorn.</li>
<li>Create sandbox environments to demo your product to customers. If Sales can run it, so can they.</li>
<li>Develop and test-drive your or other people’s Puppet modules. But see also
<a href="https://github.com/puppetlabs/beaker">beaker</a> and <a href="http://serverspec.org/">serverspec</a> if your focus is on
testing.</li>
</ul>
<h1 id="wirbelsturm-in-detail">Wirbelsturm in detail</h1>
<p>Actually I will <em>not</em> talk a whole lot about Wirbelsturm itself in this blog post anymore. If I managed to spark your
interest feel free to head over to the <a href="https://github.com/miguno/wirbelsturm">Wirbelsturm project page</a> and start
reading – and fooling around – there. There is also a list of
<a href="https://github.com/miguno/wirbelsturm#supported-puppet-modules">supported Puppet modules</a> in case you’re wondering what
kind of software you can deploy with Wirbelsturm (summary: you can use <em>any</em> Puppet module with Wirbelsturm, but some
are easier to use than others).</p>
<p>Instead I want to spend a few minutes in the next sections talking about what tasks and problems had to be solved to
put Wirbelsturm together, and also share some lessons learned along the way.</p>
<h1 id="the-long-road-of-getting-there">The long road of getting there</h1>
<p>What needed to be done to create the first version of Wirbelsturm? Here’s a non-comprehensive list, I hope my memory
serves me well.</p>
<ul>
<li>Packaging the relevant software where official packages (here: RPMs for RHEL 6 family) weren’t available.
<ul>
<li>The packaging code is also open sourced at e.g.
<a href="https://github.com/miguno/wirbelsturm-rpm-kafka">wirbelsturm-rpm-kafka</a> and
<a href="https://github.com/miguno/wirbelsturm-rpm-storm">wirbelsturm-rpm-storm</a>.</li>
<li>Of course the packages also need to be digitally signed for security reasons.</li>
<li>Kudos to Jordan Sissel for creating <a href="https://github.com/jordansissel/fpm">fpm</a>!</li>
</ul>
</li>
<li>Making this build process deterministic, and publish that code as open source, too. That is, don’t use an internal
infrastructure for that because a) people may not be easily able to reproduce it, and b) people may not trust what
strangers put together behind closed doors.
Think: <a href="http://cm.bell-labs.com/who/ken/trust.html">Reflections on Trusting Trust</a>.
<ul>
<li>The code to deploy a Wirbelsturm build server – which is used to build and sign the RPMs – is available as
open source at <a href="https://github.com/miguno/puppet-wirbelsturm_build">puppet-wirbelsturm_build</a>.</li>
</ul>
</li>
<li>Understanding how to manage and host a public yum respository on Amazon S3. <em>Please note that the idea has never</em>
<em>been to become a third-party package maintainer or third-party package repository</em>. Instead the idea was to
provide just enough so that Wirbelsturm beginners can follow a quick start and have <em>something</em> deployed in a matter
of minutes. And then let the users leverage the provided tools (see above) to run their own show.
<ul>
<li>Hosting some pre-built RPMs on a
<a href="https://github.com/miguno/puppet-wirbelsturm_yumrepos/blob/master/manifests/miguno.pp">public yum repo</a> also
meant to check whether the license of the respective software would allow that, and under which conditions. I am
not a lawyer and made my best effort to comply with all the respective licenses. If you have some concerns in
this regard please do let me know!</li>
</ul>
</li>
<li>Learning that RHEL/CentOS 6 ships with significantly outdated versions of many packages, notably
<a href="http://www.supervisord.org/">supervisord</a> (but e.g. also nginx). Supervisord version 2.x turned out to be a problem
in practice because a properly functioning process supervisor is highly recommended for running Storm & Co. in
production. Hence supervisord version 3.x needed to be packaged because that version is not yet available for the
RHEL 6 OS family in any “official” repository (e.g. EPEL’s version is outdated, too).</li>
<li>Speaking of outdated or at least different versions: Ruby on RHEL/CentOS 6 and Amazon Linux: 1.8.x. On Mac OS X
10.9: 1.9.x. And then we also have different versions of Puppet etc. While every version discrepancy is likely to
complicate development and testing, Ruby and Puppet versions were particularly annoying to deal with as they are
“bootstrap” packages that we need as the foundation of any Puppet-based deployments. I eventually created
<a href="https://github.com/miguno/ruby-bootstrap">ruby-bootstrap</a>, which addresses a part of those problems.</li>
<li>Many Puppet modules needed to be made. Where possible I tried to use existing modules as-is but in practice that
goal was hard to hit. Some modules didn’t really work, some used completely different coding styles, some did
support Hiera while others didn’t, and so on. I ended up creating several modules from scratch – e.g.
<a href="https://github.com/miguno/puppet-kafka">puppet-kafka</a>, <a href="https://github.com/miguno/puppet-storm">puppet-storm</a>, and
<a href="https://github.com/miguno/puppet-zookeeper">puppet-zookeeper</a> – as well as forking others. In the latter case,
I tried to contribute back changes to the upstream project where possible and feasible (e.g. I contributed a bug fix
to <a href="https://github.com/electrical/puppet-lib-file_concat/pull/3">puppet-lib-file_concat</a>). But because my plan was
also to come up with a consistent style and feature support across all Puppet modules – notably Hiera support –
the code of many forks stayed in that particular fork. Also, some bug fixes or features that I contributed back
upstream were never merged, but since Wirbelsturm wouldn’t function properly without those changes I didn’t have an
alternative to maintaining my own fork.</li>
<li>I ran into many bugs in many places.
<a href="stackoverflow.com/questions/17413598">Vagrant couldn’t consistently deploy to AWS</a>, for instance. Vagrant plugins
broke amidst Vagrant version upgrades.
<a href="https://github.com/mitchellh/vagrant/issues/2087">RHEL support suddenly stopped working in Vagrant</a>, which I fixed
and contributed back. I learned that Puppet has, for instance, a very weird way of handling boolean values when
defined in Hiera. Or requires you to resort to a hacky <code class="language-plaintext highlighter-rouge">mkdir -p</code> based workaround using
<a href="http://docs.puppetlabs.com/references/latest/type.html#exec">exec</a> to create directories recursively. Most of those
problems weren’t huge deals, but in combination they turned out to be death by a thousand cuts.</li>
<li>Separating Puppet code from Wirbelsturm code. I didn’t know about
<a href="https://github.com/rodjek/librarian-puppet">librarian-puppet</a> during the first early versions of Wirbelsturm, which
made it more difficult than necessary for Wirbelsturm users to keep their installations up to date. In the beginning
they needed to change Puppet code in place, i.e. files checked into the Wirbelsturm git repo, so they would often run
into merge conflicts when pulling the latest upstream changes. This unfortunate problem was resolved once I
introduced librarian-puppet.</li>
<li>Speeding up local deployments. If I recall correctly Mitchell Hashimoto – the creator of Vagrant – actually tried
parallel VM creation at some point but his (host) machine was completely overwhelmed by this, and the feature was not
introduced officially into Vagrant. However, what is still possible is to perform the <em>provisioning</em> of booted VMs
in parallel. But…the Puppet provisioner of Vagrant does not support that. I therefore created a
<a href="https://github.com/miguno/wirbelsturm/blob/master/deploy">wrapper shell script</a> based on
<a href="https://github.com/joemiller/sensu-tests/blob/master/para-vagrant.sh">para-vagrant.sh</a> so that you can benefit from
faster local deployments when using Wirbelsturm.</li>
<li>Adding support for Ansible turned out to be quick and easy, once I understood how to create
<a href="http://docs.ansible.com/intro_dynamic_inventory.html">dynamic inventory scripts</a>. 30 mins total.</li>
<li>Automating the setup steps for Amazon AWS has been tricky. Apart from so-so Vagrant support for AWS, there were a
couple of additonal problems I ran into. I remember
<a href="https://forums.aws.amazon.com/message.jspa?messageID=449984">issues with Amazon’s implementation of cloud-init</a> when
using custom AMI, for instance. Figuring out how to configure DNS in AWS (currently Wirbelsturm uses
<a href="http://aws.amazon.com/route53/">Amazon Route 53</a>) took some time.
Other tasks I remember include automatically creating restricted IAM users and tighter security groups.
I am still not perfectly happy with the Wirbelsturm user experience when deploying to AWS, and for a number of reasons
listed in the AWS related documentation of Wirbelsturm a code refactoring may be possible in the near future.</li>
<li>After reading through the various issues listed above you may also understand now why at some point I decided to
postpone supporting any other operating system than the RHEL 6 OS family (which includes CentOS and Amazon Linux).
There were simply too many moving parts, and trying to tackle e.g. Debian/Ubuntu as well might have significantly
delayed the progress on Wirbelsturm.</li>
</ul>
<h1 id="lessons-learned-mistakes-made-along-the-way">Lessons learned: mistakes made along the way</h1>
<p>The wall of shame. But hey, hindsight is 20/20.</p>
<ul>
<li>Underestimating the amount of work it eventually took. See the previous section, and even what I wrote there is not
the complete picture. Now, thanks to good roadmap planning early adopters of Wirbelsturm were productive from the
very early beginning, and a close feedback loop helped a lot to keep the project on track and running in the right
direction. Still the amount of work that actually needed to go into Wirbelsturm was significantly more than
anticipated. It wasn’t as easy as going through
<a href="http://docs.vagrantup.com/v2/provisioning/puppet_apply.html">Vagrant’s Puppet provisioner documentation</a> and writing
a few lines of Puppet code. In retrospect, knowing what I know today, Wirbelsturm could have been built <em>much</em>
faster though.</li>
<li>Not realizing quickly enough how valuable it is to separate code from configuration data in Puppet manifests, using
<a href="http://docs.puppetlabs.com/hiera/1/">Hiera</a>. Particularly because this is so second-nature when coding in “real”
programming languages instead of Puppet (which is a DSL on top of Ruby). To my defense I can only say that my
hands-on knowledge of Puppet was very limited at the beginning, and I hadn’t even heard about Hiera (and a lot of
people I talked to didn’t use it). In retrospect I should have spent more time up-front figuring out what the
Puppet ecosystem had in store to address the code-vs-data problem, because it was pretty obvious right from the
beginning that mixing the two would quickly lead to pain.</li>
<li>Adding tests to Puppet modules <em>too soon</em> and <em>too late</em>. At the beginning the Puppet modules were refactored a lot
in the quest of finding a reasonably good coding style, writing idiomatic Puppet manifests, etc. – and here dragging
unit tests along the way turned out to be a chore and a waste of time. So I stopped writing tests. While that
decision was ok, I made the mistake of postponing the re-introduction of proper tests for too long once the code
across the modules became more stable. Well, at least <a href="https://github.com/miguno/puppet-kafka">puppet-kafka</a> and
<a href="https://github.com/miguno/puppet-storm">puppet-storm</a> have a good base test setup now thanks to
<a href="https://github.com/garethr/puppet-module-skeleton">puppet-module-skeleton</a>, which means there isn’t any excuse left
to postpone adding meaningful tests.</li>
</ul>
<p>Of course there were more mistakes, but the ones above were the most noteworthy ones. :-)</p>
<h1 id="summary">Summary</h1>
<p>I am really happy that <a href="https://github.com/miguno/wirbelsturm">Wirbelsturm</a> is finally available as free and open source
software. Hopefully it will help you to quickly get up and running with technologies such as Graphite, Kafka, Storm,
Redis, and ZooKeeper. Enjoy!</p>
<p><strong>Update May 27, 2014:</strong> If you want to build real-time data processing pipelines based on Kafka and Storm, you may be
interested in <a href="/blog/2014/05/27/kafka-storm-integration-example-tutorial/">kafka-storm-starter</a>. It contains code
examples that show to integrate Apache Kafka 0.8+ with Apache Storm 0.9+, while using Apache Avro as the data
serialization format.</p>
<h1 id="related-work">Related work</h1>
<p>The following projects are similar to Wirbelsturm:</p>
<ul>
<li><a href="https://github.com/nathanmarz/storm-deploy">storm-deploy</a> – Deploys Storm clusters to AWS, by Nathan Marz, the
creator of Storm. storm-deploy has been around for much longer than Wirbelsturm, so it might be more mature. It is a
nice example of a deployment tool implemented in Clojure, using <a href="https://github.com/pallet/pallet">pallet</a> and
<a href="http://www.jclouds.org/">jclouds</a>. Because of jclouds you should also be able to deploy to clouds other than AWS,
though I haven’t found examples or documentation references on how to do so. (If you have pointers please let me
know.) Unfortunately, its Clojure roots may make storm-deploy less popular within Operations teams, who typically
are more familiar with tools such as <a href="http://puppetlabs.com/">Puppet</a>, <a href="http://www.getchef.com/">Chef</a>, or
<a href="http://www.ansible.com/">Ansible</a>. Also, storm-deploy seems to address only Storm deployments, and you require
additional tools to deploy any other infrastructure pieces that you require (or enhance storm-deploy).</li>
<li><a href="https://github.com/nathanmarz/kafka-deploy">kafka-deploy</a> – Deploys Kafka to AWS, also by Nathan Marz. It has the
same pros and cons as storm-deploy. Unfortunately, kafka-deploy has seen any updates since two years (Feb 2012),
which is around the time it was originally published.</li>
</ul>
<p>Commercial Hadoop vendors have also begun to integrate Storm into their product offerings:</p>
<ul>
<li><a href="http://hortonworks.com/hadoop/storm/">Apache Storm at HortonWorks</a> – HortonWorks are working on Storm support for
their product line. In this context they have added Storm support to their so-called
<a href="http://hortonworks.com/sandbox/">Hortonworks Sandbox</a>, which is a self-contained virtual machine with Hadoop & Co.
pre-configured.</li>
<li>If I recall correctly <a href="http://www.mapr.com/">MapR</a> were also looking at integrating Storm into their platform, but
I could not find more concrete details apart from a few
<a href="http://www.mapr.com/blog/storm-is-gearing-up-to-join-the-apache-foundation">news articles and blog posts</a>.</li>
</ul>
<p>Another way of deploying Storm is via platforms such as
<a href="https://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/YARN.html">Hadoop YARN</a> and
<a href="https://mesos.apache.org/">Apache Meos</a>:</p>
<ul>
<li><a href="https://github.com/yahoo/storm-yarn">storm-on-yarn</a> – Enables Storm clusters to be deployed into machines managed
by Hadoop YARN. The project says it is still a work in progress.</li>
<li><a href="https://github.com/nathanmarz/storm-mesos">storm-mesos</a> – Storm integration with the Mesos cluster resource manager.
The project says storm-mesos runs in production at Twitter.</li>
</ul>
<p>Lastly, there are also a few open source Puppet modules for Hadoop, Kafka, Storm, ZooKeeper & Co. I don’t want to give
an comprehensive overview of these modules in this post, but you can head over to places such as
<a href="https://forge.puppetlabs.com/">PuppetForge</a> and <a href="https://github.com/">GitHub</a> and take a look yourself. Feel free to
drop those modules into Wirbelsturm and give them a go!</p>I am happy to announce the first public release of Wirbelsturm, a Vagrant and Puppet based tool to perform 1-click local and remote deployments, with a focus on big data related infrastructure. Wirbelsturm’s goal is to make tasks such as “I want to deploy a multi-node Storm cluster” simple, easy, and fun. In this post I will introduce you to Wirbelsturm, talk a bit about its history, and show you how to launch a multi-node Storm (or Kafka or …) cluster faster than you can brew an espresso.Of Algebirds, Monoids, Monads, and other Bestiary for Large-Scale Data Analytics2013-12-02T16:45:00+01:002013-12-02T16:45:00+01:00https://www.michael-noll.com/blog/2013/12/02/twitter-algebird-monoid-monad-for-large-scala-data-analytics<p>Have you ever asked yourself what monoids and monads are, and particularly why they seem to be so attractive in the
field of large-scale data processing? Twitter recently open-sourced <a href="https://github.com/twitter/algebird">Algebird</a>,
which provides you with a JVM library to work with such algebraic data structures. Algebird is already being used in
Big Data tools such as <a href="https://github.com/twitter/scalding">Scalding</a> and
<a href="https://github.com/twitter/summingbird">SummingBird</a>, which means you can use Algebird as a mechanism to plug your
own data structures – e.g. Bloom filters, HyperLogLog – directly into large-scale data processing platforms such as
<a href="http://hadoop.apache.org/">Hadoop</a> and <a href="http://storm-project.net/">Storm</a>. In this post I will show you how to get
started with Algebird, introduce you to monoids and monads, and address the question why you should get interested in
those in the first place.</p>
<!-- more -->
<ul id="markdown-toc">
<li><a href="#goal-of-this-article" id="markdown-toc-goal-of-this-article">Goal of this article</a></li>
<li><a href="#motivating-example" id="markdown-toc-motivating-example">Motivating example</a> <ul>
<li><a href="#a-first-look-at-algebird" id="markdown-toc-a-first-look-at-algebird">A first look at Algebird</a></li>
<li><a href="#beyond-trivial-examples" id="markdown-toc-beyond-trivial-examples">Beyond trivial examples</a></li>
<li><a href="#wait-a-minute" id="markdown-toc-wait-a-minute">Wait a minute!</a></li>
<li><a href="#what-we-want-to-do" id="markdown-toc-what-we-want-to-do">What we want to do</a></li>
</ul>
</li>
<li><a href="#my-journey-down-the-rabbit-hole" id="markdown-toc-my-journey-down-the-rabbit-hole">My journey down the rabbit hole</a> <ul>
<li><a href="#how-this-post-started" id="markdown-toc-how-this-post-started">How this post started</a></li>
<li><a href="#scala-functors-monoids-monads-category-theory-implicits-type-classes-aaargh" id="markdown-toc-scala-functors-monoids-monads-category-theory-implicits-type-classes-aaargh">Scala, functors, monoids, monads, category theory, implicits, type classes, aaargh!</a></li>
</ul>
</li>
<li><a href="#the-tldr-version-of-monoids-and-monads" id="markdown-toc-the-tldr-version-of-monoids-and-monads">The TL;DR version of monoids and monads</a> <ul>
<li><a href="#monoids" id="markdown-toc-monoids">Monoids</a> <ul>
<li><a href="#what-is-a-monoid" id="markdown-toc-what-is-a-monoid">What is a monoid?</a></li>
<li><a href="#monoids-in-more-detail" id="markdown-toc-monoids-in-more-detail">Monoids in more detail</a></li>
<li><a href="#what-are-example-monoids" id="markdown-toc-what-are-example-monoids">What are example monoids?</a></li>
<li><a href="#what-can-i-use-a-monoid-for--why-should-i-look-for-one" id="markdown-toc-what-can-i-use-a-monoid-for--why-should-i-look-for-one">What can I use a monoid for? Why should I look for one?</a></li>
</ul>
</li>
<li><a href="#monads" id="markdown-toc-monads">Monads</a> <ul>
<li><a href="#what-is-a-monad" id="markdown-toc-what-is-a-monad">What is a monad?</a></li>
<li><a href="#monads-in-more-detail" id="markdown-toc-monads-in-more-detail">Monads in more detail</a></li>
<li><a href="#what-are-example-monads" id="markdown-toc-what-are-example-monads">What are example monads?</a></li>
<li><a href="#what-can-i-use-a-monad-for--why-should-i-look-for-one" id="markdown-toc-what-can-i-use-a-monad-for--why-should-i-look-for-one">What can I use a monad for? Why should I look for one?</a></li>
</ul>
</li>
</ul>
</li>
<li><a href="#algebird" id="markdown-toc-algebird">Algebird</a> <ul>
<li><a href="#creating-a-monoid" id="markdown-toc-creating-a-monoid">Creating a monoid</a> <ul>
<li><a href="#the-twitteruser-type" id="markdown-toc-the-twitteruser-type">The TwitterUser type</a></li>
<li><a href="#the-maxtwitteruser-monoid" id="markdown-toc-the-maxtwitteruser-monoid">The Max[TwitterUser] monoid</a></li>
<li><a href="#where-to-go-from-here" id="markdown-toc-where-to-go-from-here">Where to go from here?</a></li>
</ul>
</li>
<li><a href="#creating-a-monad" id="markdown-toc-creating-a-monad">Creating a monad?</a></li>
<li><a href="#key-algebraic-structures-in-algebird" id="markdown-toc-key-algebraic-structures-in-algebird">Key algebraic structures in Algebird</a></li>
<li><a href="#a-small-algebird-faq" id="markdown-toc-a-small-algebird-faq">A small Algebird FAQ</a> <ul>
<li><a href="#error-cannot-find-groupmonoid-type-class-for-a-type-t" id="markdown-toc-error-cannot-find-groupmonoid-type-class-for-a-type-t">Error “Cannot find Group/Monoid/… type class for a type T”?</a></li>
<li><a href="#combine-different-monoids" id="markdown-toc-combine-different-monoids">Combine different monoids?</a></li>
</ul>
</li>
</ul>
</li>
<li><a href="#are-monads-really-everywhere" id="markdown-toc-are-monads-really-everywhere">Are monads really everywhere?</a></li>
<li><a href="#summary" id="markdown-toc-summary">Summary</a></li>
<li><a href="#references" id="markdown-toc-references">References</a> <ul>
<li><a href="#monads-and-monoids" id="markdown-toc-monads-and-monoids">Monads and monoids</a></li>
<li><a href="#summingbird" id="markdown-toc-summingbird">SummingBird</a></li>
<li><a href="#category-theory" id="markdown-toc-category-theory">Category theory</a></li>
</ul>
</li>
</ul>
<h1 id="goal-of-this-article">Goal of this article</h1>
<p>The main goal of this is article is to spark your <em>curiosity</em> and <em>motivation</em> for
<a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monad.scala">Algebird</a>
and the concepts of monoid, monads, and category theory in general. In other words, I want to address the questions
<em>“What’s the big deal? Why should I care? And how can these theoretical concepts help me in my daily work?”</em></p>
<p>While I will explain a little bit what the various concepts such as monoids are, this is not the focus of this post.
If in doubt I will rather err on the side of grossly oversimplifying a topic to get the point across even at the
expense of correctness. There are much better resources available online and offline that can teach you the full
details of the various items I will discuss here. That being said, I compiled a list of references at the end of this
article so that you have a starting point to understand the following concepts in full detail, and with more accurate
and thorough explanations than I could come up with.</p>
<h1 id="motivating-example">Motivating example</h1>
<h2 id="a-first-look-at-algebird">A first look at Algebird</h2>
<p>Here is a simple example what you can do with monoids and monads, based on the starter example in
<a href="https://github.com/twitter/algebird/">Algebird</a>.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="n">scala</span><span class="o">></span> <span class="nc">Max</span><span class="o">(</span><span class="mi">10</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="mi">30</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="mi">20</span><span class="o">)</span>
<span class="n">res1</span><span class="k">:</span> <span class="kt">com.twitter.algebird.Max</span><span class="o">[</span><span class="kt">Int</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Max</span><span class="o">(</span><span class="mi">30</span><span class="o">)</span>
<span class="c1">// Alternative, Java-like (read: ugly) syntax for readers unfamiliar with Scala.</span>
<span class="n">scala</span><span class="o">></span> <span class="nc">Max</span><span class="o">(</span><span class="mi">10</span><span class="o">).+(</span><span class="nc">Max</span><span class="o">(</span><span class="mi">30</span><span class="o">)).+(</span><span class="nc">Max</span><span class="o">(</span><span class="mi">20</span><span class="o">))</span>
<span class="n">res2</span><span class="k">:</span> <span class="kt">com.twitter.algebird.Max</span><span class="o">[</span><span class="kt">Int</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Max</span><span class="o">(</span><span class="mi">30</span><span class="o">)</span>
</code></pre></div></div>
<p>What is happening here? Basically, we are boxing two numbers, the <code class="language-plaintext highlighter-rouge">Int</code> values <code class="language-plaintext highlighter-rouge">4</code> and <code class="language-plaintext highlighter-rouge">5</code>, into <code class="language-plaintext highlighter-rouge">Max</code>, and then
we are “adding” them. The behavior of <code class="language-plaintext highlighter-rouge">Max[T]</code> turns the <code class="language-plaintext highlighter-rouge">+</code> operator into a function that returns the largest boxed
<code class="language-plaintext highlighter-rouge">T</code>.</p>
<p>Conceptually this is similar to the following native Scala code:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// This is native Scala.</span>
<span class="n">scala</span><span class="o">></span> <span class="mi">10</span> <span class="n">max</span> <span class="mi">30</span> <span class="n">max</span> <span class="mi">20</span>
<span class="n">res3</span><span class="k">:</span> <span class="kt">Int</span> <span class="o">=</span> <span class="mi">30</span>
<span class="c1">// Alternative, Java-like syntax.</span>
<span class="n">scala</span><span class="o">></span> <span class="mf">10.</span><span class="nf">max</span><span class="o">(</span><span class="mi">30</span><span class="o">).</span><span class="py">max</span><span class="o">(</span><span class="mi">20</span><span class="o">)</span>
<span class="n">res4</span><span class="k">:</span> <span class="kt">Int</span> <span class="o">=</span> <span class="mi">30</span>
</code></pre></div></div>
<p>At this point you may ask, “Alright, what is the big deal? The native Scala example looks actually better!”</p>
<p>At least, that is what I thought myself at first. But the simplicity of this example is deceptive. There is a lot
more to it than meets the eye at first sight.</p>
<h2 id="beyond-trivial-examples">Beyond trivial examples</h2>
<p>Admittedly, the first example used a very dull data structure, <code class="language-plaintext highlighter-rouge">Int</code>. Any programming language comes with built-in
functionality to add two integers, right? So you would be hardly convinced of the value of a tool like Algebird if all
it allowed you to do was <code class="language-plaintext highlighter-rouge">4 + 3 = 7</code>, particularly when doing those simple things would require you to understand
sophisticated concepts such as monoids and monads. Too much effort for too little value I would say!</p>
<p>So let me use a different example because adding <code class="language-plaintext highlighter-rouge">Int</code> values is indeed trivial. Imagine that you are working on
large-scale data analytics that make heavy use of <a href="https://en.wikipedia.org/wiki/Bloom_filter">Bloom filters</a>. Your
applications are based on highly-parallel tools such as Hadoop or Storm, and they create and work with many such Bloom
filters in parallel. Now the money question is: <em>How do you combine or add two Bloom filters in an easy way?</em>
(This is where monoids come into play.)</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">first</span> <span class="k">=</span> <span class="nc">BloomFilter</span><span class="o">(...)</span>
<span class="k">val</span> <span class="nv">second</span> <span class="k">=</span> <span class="nc">BloomFilter</span><span class="o">(...)</span>
<span class="n">first</span> <span class="o">+</span> <span class="n">second</span> <span class="o">==</span> <span class="n">uh</span><span class="o">?</span>
</code></pre></div></div>
<p>And what about performing other operations on those Bloom filter instances, notably <em>data processing pipelines</em> based on
common functions such as <code class="language-plaintext highlighter-rouge">map</code>, <code class="language-plaintext highlighter-rouge">flatMap</code>, <code class="language-plaintext highlighter-rouge">foldLeft</code>, <code class="language-plaintext highlighter-rouge">reduceLeft</code>? (And this is where monads come to into play.)</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">val</span> <span class="nv">filters</span> <span class="k">=</span> <span class="nc">Seq</span><span class="o">[</span><span class="kt">BloomFilter</span><span class="o">](...)</span>
<span class="k">val</span> <span class="nv">summary</span> <span class="k">=</span> <span class="n">filters</span> <span class="n">flatMap</span> <span class="o">{</span> <span class="cm">/* magic happens here */</span> <span class="o">}</span> <span class="n">reduceLeft</span> <span class="o">{</span> <span class="cm">/* more magic */</span> <span class="o">}</span> <span class="o">...</span>
</code></pre></div></div>
<p>And what about combining two
<a href="http://blog.aggregateknowledge.com/2012/10/25/sketch-of-the-day-hyperloglog-cornerstone-of-a-big-data-infrastructure/">HyperLogLog</a>
instances?</p>
<p>Intuitively we could say that the general idea of “adding” two Bloom filters is quite similar to how we would add two
sets <em>A</em> and <em>B</em>, where adding would mean creating the union set of <em>A</em> and <em>B</em>.</p>
<p>Now Algebird addresses this problem of abstraction. In a nutshell, if you can turn a data structure into a monoid
(or semigroup, or …), then Algebird allows you to put it to good use. You can then work with your data structure
just as nicely as you are so used to when dealing with <code class="language-plaintext highlighter-rouge">Int</code>, <code class="language-plaintext highlighter-rouge">Double</code> or <code class="language-plaintext highlighter-rouge">List</code>. And you can use it with large-scale
data processing tools such as Hadoop and Storm, too.</p>
<h2 id="wait-a-minute">Wait a minute!</h2>
<p>In case you are asking yourself the following question (which I did): Is the magic of Algebird simply something like a
custom <code class="language-plaintext highlighter-rouge">Max[Int]</code> class that defines a <code class="language-plaintext highlighter-rouge">+()</code> method, similar to the following snippet but actually with a bounded
type parameter <code class="language-plaintext highlighter-rouge">T : Ordering[T]</code>? (If you do not understand the latter, take a look at
<a href="http://stackoverflow.com/questions/17597961">this StackOverflow thread</a>.)</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Is Algebird implemented like this? (hint: nope)</span>
<span class="n">scala</span><span class="o">></span> <span class="k">case</span> <span class="k">class</span> <span class="nc">Max</span><span class="o">(</span><span class="k">val</span> <span class="nv">i</span><span class="k">:</span> <span class="kt">Int</span><span class="o">)</span> <span class="o">{</span> <span class="k">def</span> <span class="nf">+</span><span class="o">(</span><span class="n">that</span><span class="k">:</span> <span class="kt">Max</span><span class="o">)</span> <span class="k">=</span> <span class="nf">if</span> <span class="o">(</span><span class="k">this</span><span class="o">.</span><span class="py">i</span> <span class="o">>=</span> <span class="nv">that</span><span class="o">.</span><span class="py">i</span><span class="o">)</span> <span class="k">this</span> <span class="k">else</span> <span class="n">that</span> <span class="o">}</span>
<span class="n">defined</span> <span class="k">class</span> <span class="nc">Max</span>
<span class="n">scala</span><span class="o">></span> <span class="nc">Max</span><span class="o">(</span><span class="mi">10</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="mi">30</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="mi">20</span><span class="o">)</span>
<span class="n">res5</span><span class="k">:</span> <span class="kt">Max</span> <span class="o">=</span> <span class="nc">Max</span><span class="o">(</span><span class="mi">30</span><span class="o">)</span>
</code></pre></div></div>
<p>The answer is yes and no. “Yes” because it <em>is</em> similar. And “no” because the implementation is quite different from
the above analogy, and provides you with significantly more algebra-fu (but again, it has the same spirit).</p>
<h2 id="what-we-want-to-do">What we want to do</h2>
<p>Our goal in this post is to build a data structure <code class="language-plaintext highlighter-rouge">TwitterUser</code> accompanied by a <code class="language-plaintext highlighter-rouge">Max[TwitterUser]</code> monoid view of it.
We want to use the two for implementing the analytics of a fictional popularity contest on Twitter, like so:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Let's have a popularity contest on Twitter. The user with the most followers wins!</span>
<span class="k">val</span> <span class="nv">barackobama</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"BarackObama"</span><span class="o">,</span> <span class="mi">40267391</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">katyperry</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"katyperry"</span><span class="o">,</span> <span class="mi">48013573</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">ladygaga</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"ladygaga"</span><span class="o">,</span> <span class="mi">40756470</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">miguno</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"miguno"</span><span class="o">,</span> <span class="mi">731</span><span class="o">)</span> <span class="c1">// I participate, too. Olympic spirit!</span>
<span class="k">val</span> <span class="nv">taylorswift</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"taylorswift13"</span><span class="o">,</span> <span class="mi">37125055</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">winner</span><span class="k">:</span> <span class="kt">Max</span><span class="o">[</span><span class="kt">TwitterUser</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Max</span><span class="o">(</span><span class="n">barackobama</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">katyperry</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">ladygaga</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">miguno</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">taylorswift</span><span class="o">)</span>
<span class="nf">assert</span><span class="o">(</span><span class="nv">winner</span><span class="o">.</span><span class="py">get</span> <span class="o">==</span> <span class="n">katyperry</span><span class="o">)</span>
</code></pre></div></div>
<p>Figuring out how to do this with monoids, monads, and Algebird is the objective of this article.</p>
<p>Of course, instead of using Algebird and monoids we could also project the number-of-followers field from each user
and perform any such analytics directly on the <code class="language-plaintext highlighter-rouge">Int</code> values. That’s not the point however. I intentionally wanted a
very simple example use case because, as you will see, there is so much to understand about what’s going on behind the
scenes that any further distraction should be avoided. At least, that was my personal experience. :-)</p>
<h1 id="my-journey-down-the-rabbit-hole">My journey down the rabbit hole</h1>
<p><em>This section is more for entertainment. Feel free to skip it.</em></p>
<h2 id="how-this-post-started">How this post started</h2>
<p>I am following a few Twitter folks on, well, Twitter such as Dmitriy Ryaboy
(<a href="https://twitter.com/squarecog">@squarecog</a>) and Oscar Boykin (<a href="https://twitter.com/posco">@posco</a>). And lately
they talked a lot about how data analytics at Twitter is powered by “monoids” and “monads”, and how tools such as
<a href="https://github.com/twitter/algebird/">Algebird</a> and <a href="https://github.com/twitter/scalding">Scalding</a> form the
code foundation of their analytics infrastructure.</p>
<p>Here is an example of such a conversation:</p>
<p><a href="http://makeameme.org/media/created/Monads-monads-everywhere.jpg"><img src="https://www.michael-noll.com/assets/uploads/twitter-monad-conversation.png" alt="Monads. Monads everywhere!" /></a></p>
<p>A <em>mo</em>-what? And how comes those things are apparently spreading like a contagious disease throughout their data
analytics code?</p>
<p>Another trigger was a discussion involing Ted Dunning of MapR (<a href="https://twitter.com/ted_dunning">@ted_dunning</a>) and
his work on a new data structure called <a href="https://github.com/tdunning/t-digest">t-digest</a>:</p>
<p><img src="https://www.michael-noll.com/assets/uploads/twitter-associative-conversation.png" alt="Is t-digest associative?" /></p>
<p>Why was Ted being asked whether <code class="language-plaintext highlighter-rouge">t-digest</code> is associative? And how does all this relate to semigroups and monoids? And
finally, what the heck are semigroups in the first place?</p>
<p>Now a dangerous series of events began to take place on my side.</p>
<p>First I thought, <em>“Hey, coincidentally I have started to pick up Scala around a month ago. Given that Algebird is</em>
<em>written in Scala this might turn into an interesting finger exercise.”</em> (Note my focus on “finger exercise”.)
On top of that I knew that the use of Algebird extends to other interesting big data tools such as Storm and Scalding,
so it could turn out that I would not only learn something for learning’s sake but that I could put it to practical use
in my daily work, too. The combination of these two factors – general interest and practical applicability –
eventually caused me to give in to my curiosity and decided to put “an hour or two aside” to read up on those monoid
thingies and figure out whether and how I could leverage Algebird for my own purposes.</p>
<p>You might notice at this point that it all started quite innocently. But what I did not realize at that moment was that
I was opening <a href="https://en.wikipedia.org/wiki/Pandora%27s_box">Pandora’s box</a> on an otherwise quiet and peaceful
Swiss weekend…</p>
<h2 id="scala-functors-monoids-monads-category-theory-implicits-type-classes-aaargh">Scala, functors, monoids, monads, category theory, implicits, type classes, aaargh!</h2>
<p>What started as a seemingly innocent journey down a calm park lane quickly turned into the opening of the gates of
functional programming and category theory hell. Not only did I struggle to understand what things like functors,
semigroups, monoids, and other algebraic structures that only a mother could love are. No, on top of that I quickly
realized that how these things can be implemented in Scala in general and in Algebird in particular meant I had to take
my beginner Scala-fu to a whole new level. In the end it took me the full weekend to grasp all those concepts to the
point where I’d say right now that I know enough to be dangerous.</p>
<p>The learning curve reminded me a lot of the following famous picture:</p>
<p><img src="https://www.michael-noll.com/assets/uploads/editor-learning-curve.png" alt="Learning curve for some common editors" /></p>
<div class="caption">
Figure 1: Learning curve for some common editors. Image courtesy of <a href="https://josem.co/learning-vim-introduction/">Jose M. Gilgado</a>.
</div>
<p>And it did feel like the <code class="language-plaintext highlighter-rouge">vi</code> curve – the brick wall experience. What else could it be, right? That being said I
still fear that, after having hit and finally made it over that initial brick wall, it may still spiral out of control
again like the Emacs curve. :-)</p>
<p>Picture myself sitting in front of my keyboard, frantically interacting with your favorite search engine, StackOverflow,
Wikipedia, your usual suspects of Scala books, and what not:</p>
<p>Me: “What is a monad?”</p>
<p><em>Internet: “A <a href="http://stackoverflow.com/questions/3870088">monad is just a monoid in the category of endofunctors</a>.”</em></p>
<p>Me: “Hmm, ok. So what is a monoid?”</p>
<p><em>Internet: “A monoid is a semigroup with identity.”</em></p>
<p>Me: “Then what is a semigroup??” (number of question marks increases with anxiety level)</p>
<p><em>Internet: “An algebraic structure consisting of a set together with an associative binary operation.”</em></p>
<p>Me: “Alright, I see the mathematical definition and I do see a soup of greek letters. Still, what <em>is</em> it?
<a href="http://codahale.com/downloads/email-to-donald.txt">Where can I get one from, and what can I use it for?</a>”</p>
<p><em>Internet: “Here is an example in the Haskell programming language.”</em></p>
<p>Me: <censored></p>
<p>On a more serious note, the past few days have really been a tour de force where I felt I would recursively dive from
one new term or concept into yet more new terms and concepts, to the point where my brain would run into a stack
overflow. <em>“Why am I actually reading about <a href="https://en.wikipedia.org/wiki/Magma_%28algebra%29">magmas</a>, or co- and</em>
<em>contra-variance in Scala, or bounded type parameters? What was the original question I tried to find an answer for?”</em></p>
<p>To make a long story short I was really deep down the rabbit hole, with no Alice in sight but fully surrounded by
semigroups of monoidal and diabolical <a href="http://en.wikipedia.org/wiki/Jabberwocky">jabberwockies</a> on a big night out.
Given the questions, comments and blog posts of other folks at least I found consolation in the fact that I was
apparently not alone.</p>
<p>And, finally, at the end of the hole there was a bit of light. In the next sections I want to share what I have learned
so far in the hope that it will prove helpful for you, too. We start with a brief introduction to monoids and monads,
followed by how to apply what we have learned in Algebird hands-on.</p>
<h1 id="the-tldr-version-of-monoids-and-monads">The TL;DR version of monoids and monads</h1>
<blockquote>
<p><em>Michael’s abridged relation of monoids and monads:</em>
A monad is a monoid where you blend the “oi” into an “a”. Depending on your typesettings (pun intended) this blend
will be easier or harder for you to see. If in doubt, squint more.</p>
</blockquote>
<p>As a grossly simplified rule of thumb:</p>
<ol>
<li><strong>Monoid</strong>: If you want to “attach” <em>operations</em> such as <code class="language-plaintext highlighter-rouge">+</code>, <code class="language-plaintext highlighter-rouge">-</code>, <code class="language-plaintext highlighter-rouge">*</code>, <code class="language-plaintext highlighter-rouge">/</code> or <code class="language-plaintext highlighter-rouge"><=</code> to <em>data objects</em> – say, adding
two Bloom filters – then you want to provide <em>monoid</em> forms for those data objects (e.g. a monoid for your Bloom
filter data structure). This way you can combine and juggle your custom data structures just like you would do with
plain integer numbers.</li>
<li><strong>Monad</strong>: If you want to create <em>data processing pipelines</em> that turn data objects step-by-step into the desired,
final output (e.g. aggregating raw records into summary statistics), then you want to build one or more <em>monads</em> to
model these data pipelines. Particularly if you want to run those pipelines in large-scala data processing platforms
such as Hadoop or Storm.</li>
</ol>
<p>The intent of this section is to give you a high-level idea what those concepts are, and what you can use them for.
That is, this section should help you determine whether you want to venture down the rabbit hole, too.</p>
<p>I did not want to add yet another variant to the pool of “what is a monoid/monad” articles, but at the same time I felt
I need to explain at least very briefly what the various concepts are (as good as I can) so that you can better
understand how to use a tool such as Algebird.</p>
<p>Of course, if you ran across a blatant mistake on my side please do let me know!</p>
<h2 id="monoids">Monoids</h2>
<h3 id="what-is-a-monoid">What is a monoid?</h3>
<p>A monoid is a structure that consists of:</p>
<ol>
<li>a set of objects (such as numbers)</li>
<li>a binary operation as a method of combining them (such as adding those numbers)</li>
</ol>
<p>The small catch is that the way you can combine the objects in your set must adhere to a few rules, which are described
in the next section.</p>
<p>One way to explain a monoid in the context of programming is as a kind of <em>adapter</em> or <em>bounded view</em> of a type <code class="language-plaintext highlighter-rouge">T</code>.
Imagine a data structure of type <code class="language-plaintext highlighter-rouge">T</code> – say, a <code class="language-plaintext highlighter-rouge">List</code>. If you can find a way to use <code class="language-plaintext highlighter-rouge">T</code> in a way that conforms to the
monoid laws (see next section), then you can say “type T forms a monoid” <code class="language-plaintext highlighter-rouge">Monoid[T]</code>; for instance, if the binary
operation you picked behaves like the concept of addition, you have an additive monoid view of <code class="language-plaintext highlighter-rouge">T</code>.</p>
<div class="note">
Note: What I tried to highlight in the previous paragraph is that a given type <tt>T</tt> can have multiple monoidal
forms. An additive monoid of <tt>T</tt> is just an example, and <tt>T</tt> might have more monoids than the additive
variant. Also -- sorry for the forward reference -- a type <tt>T</tt> can form both a monoid <em>and</em> a monad.
One such dual-headed hydra is the well-known <tt>List</tt>.
</div>
<p>So you can read <code class="language-plaintext highlighter-rouge">Monoid[T]</code> as <em>“T looks like a monoid and quacks like a monoid, so it must be a monoid”</em>. This notion
is related to the concept of <a href="http://en.wikipedia.org/wiki/Duck_typing">duck typing</a> in languages such as Python.
Scala, in which Algebird is implemented, has a static type system though, and to achieve such ad-hoc polymorphism we
typically use
<a href="http://danielwestheide.com/blog/2013/02/06/the-neophytes-guide-to-scala-part-12-type-classes.html">type classes</a>
to achieve a similar effect. A nifty feature of type classes is that they allow you to retroactively add polymorphism
even to existing types that are not under your own control: examples are <code class="language-plaintext highlighter-rouge">Seq</code> or <code class="language-plaintext highlighter-rouge">List</code>, which are provided by the
Scala standard library and thus not under your control.</p>
<p><img src="https://www.michael-noll.com/assets/uploads/monoid-illustration.png" alt="Monoid illustration" /></p>
<div class="caption">
Figure 2: A monoid seen as a bounded view. In this analogy we are looking at the original type <tt>T</tt> from a
different, "monoidal angle". Here, we are combining two values of type <tt>T</tt> under the laws of the pink-colored
monoid view of <tt>T</tt> (whatever this particular monoid might actually be doing).
</div>
<h3 id="monoids-in-more-detail">Monoids in more detail</h3>
<p>A monoid is a set of objects, <code class="language-plaintext highlighter-rouge">T</code>, together with a binary operation <tt>⋅</tt> that satisfies the three axioms
listed below.</p>
<p>One way to express a monoid in Scala would be the following trait, used as a type class:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Important: What you see here is only part of the contract.</span>
<span class="c1">// The monoid, and thus `e` and `o`, must also adhere to the monoid laws.</span>
<span class="k">trait</span> <span class="nc">Monoid</span><span class="o">[</span><span class="kt">T</span><span class="o">]</span> <span class="o">{</span>
<span class="k">def</span> <span class="nf">e</span><span class="k">:</span> <span class="kt">T</span>
<span class="k">def</span> <span class="nf">op</span><span class="o">(</span><span class="n">a</span><span class="k">:</span> <span class="kt">T</span><span class="o">,</span> <span class="n">b</span><span class="k">:</span> <span class="kt">T</span><span class="o">)</span><span class="k">:</span> <span class="kt">T</span>
<span class="o">}</span>
</code></pre></div></div>
<dl>
<dt>Closure</dt>
<dd>
For all <em>a, b</em> in <em>T</em>, the result of the operation <em>a ⋅ b</em> is also in <em>T</em>:
$$
\forall a,b \in T: a \bullet b \in T
$$
In Scala, we could express this axiom with the following function signature for <tt>⋅</tt>:
<tt>def op(a: T, b: T): T</tt>
</dd>
<dt>Associativity</dt>
<dd>
For all <em>a</em>, <em>b</em>, and <em>c</em> in <em>T</em>,
the equation <em>(a ⋅ b) ⋅ c = a ⋅ (b ⋅ c)</em> holds:
$$
\forall a,b,c \in T: (a \bullet b) \bullet c = a \bullet (b \bullet c)
$$
In Scala, we could express this axiom with:
<tt>(a op b) op c == a op (b op c)</tt>
</dd>
<dt>Identity element</dt>
<dd>
There exists an element <em>e</em> (we could also call it <em>zero</em> to draw a link to addition) in <em>T</em>,
such that for all elements <em>a</em> in <em>T</em>, the equation <em>e ⋅ a = a ⋅ e = a</em> holds:
$$
\exists e \in T: \forall a \in T: e \bullet a = a \bullet e = a
$$
In Scala, we could express this axiom with the following, which as you might note captures the idea of a
<a href="http://en.wikipedia.org/wiki/NOP">no-op</a>: <tt>e op a == a op e == a</tt>
</dd>
</dl>
<div class="note">
Note: <em>Any</em> binary operation satisfying the three axioms above qualifies your data structure to be a monoid.
It does not necessarily need to be an addition-like operation.
</div>
<p>Before we move on and look at examples of monads, I want to mention one more thing about the binary function of a
monoid. We have learned that it must be <a href="http://en.wikipedia.org/wiki/Associative_property">associative</a>. Wouldn’t it
be helpful if the binary function were <a href="http://en.wikipedia.org/wiki/Commutative_property">commutative</a>, too, even
though this optional feature would not be required to make a monoid?</p>
<p>Here is a transcripted reply during Sam Ritchie’s SummingBird talk at CUFP:</p>
<blockquote>
<p>Question: Associativity is one nice thing about monoids, but what about commutativity [which] is also important. Are there examples of non-commutative datastructures</p>
<p>Answer: It should be baked into the algebra (non-commutativity). This helps with data skew in particular. An important non-commutative application is Twitter itself! When you want to build the List monoid, the key is <tt>userid,time</tt> and the value is the list of tweets over that timeline (so ordering matters here). It’s not good to get a non-deterministic order when building up these lists in parallel, so that’s a good example of when associativity and commutativity are both important.</p>
<p>Source: <a href="http://www.syslog.cl.cam.ac.uk/2013/09/22/liveblogging-cufp--2013/">Transcript of Sam Ritchie’s SummingBird talk at CUFP 2013</a></p>
</blockquote>
<h3 id="what-are-example-monoids">What are example monoids?</h3>
<ul>
<li><em>Numbers</em> (= the set of objects) you can <em>add</em> (= the method of combining them).
<ul>
<li>For integer addition, <code class="language-plaintext highlighter-rouge">e == 0</code> and <code class="language-plaintext highlighter-rouge">op == +</code>.</li>
<li>For integer multiplication, <code class="language-plaintext highlighter-rouge">e == 1</code> and <code class="language-plaintext highlighter-rouge">op == *</code>.</li>
</ul>
</li>
<li><em>Lists</em> you can <em>concatenate</em>.
<ul>
<li>With <code class="language-plaintext highlighter-rouge">e == Nil</code> and <code class="language-plaintext highlighter-rouge">op == concat</code>.</li>
</ul>
</li>
<li><em>Sets</em> you can <em>union</em>.
<ul>
<li>With <code class="language-plaintext highlighter-rouge">e == Set()</code> and <code class="language-plaintext highlighter-rouge">op == union</code>.</li>
</ul>
</li>
</ul>
<p>There are more and also more sophisticated examples, of course. <code class="language-plaintext highlighter-rouge">Max[Int]</code> at the beginning of this article is a
monoid, too.</p>
<p>Here is how Algebird defines an <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monoid.scala">additive monoid for the standard type <code class="language-plaintext highlighter-rouge">Seq</code></a>:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// A `Seq` concatenation monoid.</span>
<span class="c1">// Plus (the `op`) means concatenation,</span>
<span class="c1">// zero (the identity element `e`) is the empty Seq.</span>
<span class="k">class</span> <span class="nc">SeqMonoid</span><span class="o">[</span><span class="kt">T</span><span class="o">]</span> <span class="nc">extends</span> <span class="nc">Monoid</span><span class="o">[</span><span class="kt">Seq</span><span class="o">[</span><span class="kt">T</span><span class="o">]]</span> <span class="o">{</span>
<span class="k">override</span> <span class="k">def</span> <span class="nf">zero</span> <span class="k">=</span> <span class="nc">Seq</span><span class="o">[</span><span class="kt">T</span><span class="o">]()</span>
<span class="k">override</span> <span class="k">def</span> <span class="nf">plus</span><span class="o">(</span><span class="n">left</span> <span class="k">:</span> <span class="kt">Seq</span><span class="o">[</span><span class="kt">T</span><span class="o">],</span> <span class="n">right</span> <span class="k">:</span> <span class="kt">Seq</span><span class="o">[</span><span class="kt">T</span><span class="o">])</span> <span class="k">=</span> <span class="n">left</span> <span class="o">++</span> <span class="n">right</span>
<span class="o">}</span>
<span class="c1">// Make an instance of `SeqMonoid` available as an implicit value.</span>
<span class="c1">// This is a Scala-specific implementation action that needs to be done,</span>
<span class="c1">// i.e. it is not related to the abstract concept of monoids.</span>
<span class="c1">//</span>
<span class="c1">// The effect of this statement is to add the "monoid view" of Seq</span>
<span class="c1">// as defined above to all `Seq` instances in the code. If you</span>
<span class="c1">// define your own monoid for a type `T` in Algebird and forget</span>
<span class="c1">// this statement, Algebird will complain with the following</span>
<span class="c1">// @implicitNotFound error message:</span>
<span class="c1">//</span>
<span class="c1">// "Cannot find Monoid type class for T"</span>
<span class="c1">//</span>
<span class="c1">// Implicits need to be used because this is how the notion of</span>
<span class="c1">// type classes is implemented in Scala.</span>
<span class="k">implicit</span> <span class="k">def</span> <span class="nf">seqMonoid</span><span class="o">[</span><span class="kt">T</span><span class="o">]</span> <span class="k">:</span> <span class="kt">Monoid</span><span class="o">[</span><span class="kt">Seq</span><span class="o">[</span><span class="kt">T</span><span class="o">]]</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">SeqMonoid</span><span class="o">[</span><span class="kt">T</span><span class="o">]</span>
</code></pre></div></div>
<p>Algebird actually includes a few more methods for the <code class="language-plaintext highlighter-rouge">Monoid[T]</code> type class – which <code class="language-plaintext highlighter-rouge">SeqMonoid[T]</code> extends – but the
key functionality is shown above.</p>
<h3 id="what-can-i-use-a-monoid-for--why-should-i-look-for-one">What can I use a monoid for? Why should I look for one?</h3>
<p>Whenever you have a data structure (which backs your “set of objects”, e.g. the <code class="language-plaintext highlighter-rouge">Int</code> data structure or the <code class="language-plaintext highlighter-rouge">List[T]</code>
data structure) you can begin checking whether you can define one or more monoids for that data structure. Here you
will start looking for operations you can perform on any two instances of your data structure that satisfy the three
<a href="https://en.wikipedia.org/wiki/Monoid">monoid axioms</a>: <em>closure</em>, <em>associativity</em>, and <em>identity element</em> (the latter
gives your monoid a no-op function, and is the one thing that turns a semigroup into a monoid).</p>
<p>If you do find any such monoids for your data structure, hooray! On the practical side this means means that you can
now use your data structure in any code that expects a monoid. As I said above you can think of a monoid as an adapter,
or shape, for (some monoid-compatible aspects of) your data structure that allows you to fit your data structure peg
into a monoid hole. Some such holes are <a href="https://github.com/twitter/algebird/">Algebird</a>,
<a href="https://github.com/twitter/scalding">Scalding</a> and <a href="https://github.com/twitter/summingbird">Summingbird</a> of Twitter.
Being supported by those tools also means that you can now plug your data structure into big data analytics tools such
as Hadoop and Storm, which might be a huge seller and productivity gain for your new data structure.</p>
<p><span class="pullquote-right">If your data structure has a monoid form, this means you can plug the data structure directly into large-scale data
processing platforms such as Hadoop and Storm.</span>
Secondly, and in more general terms, it signifies because of the <em>associativity</em> of monoid operations that those
operations on your data structure <a href="http://en.wikipedia.org/wiki/Monoid#Monoids_in_computer_science">can be parallelized</a>
in order to utilize multiple CPU cores efficiently. Speaking in code, that means you can run operations such as
<code class="language-plaintext highlighter-rouge">foldLeft()</code> and <code class="language-plaintext highlighter-rouge">reduceLeft()</code> on them. And parallelization support is yet another reason why monoids (and monads) are
so attractive for big data tools such as Hadoop and Storm, where your code not only runs on many cores per machine but
on many such machines in a cluster. In other words:
If your data structure has a monoid form, this means you can plug the data structure directly into large-scale data
processing platforms such as Hadoop and Storm.
Hence monoids enable you to <a href="http://en.wikipedia.org/wiki/MapReduce">MapReduce</a> and to
<a href="http://en.wikipedia.org/wiki/Divide_and_conquer_algorithm">divide and conquer</a>.</p>
<p>Let me quote Sam Ritchie (<a href="https://twitter.com/sritchie">@sritchie</a>), former Twitter engineer and now founder of
<a href="http://www.paddleguru.com/">PaddleGuru</a> (cool idea, by the way – go sports!) for a very concrete practical application
of monoids at Twitter. Well, actually I am quoting a transcript of his talk.</p>
<blockquote>
<p>One cool feature: When you visit a tweet, you want the reverse feed of things that have embedded the tweet. The MapReduce graph for this comes from: When you see an impression, find the key of the tweet and emit a tuple of the <tt>tweetId</tt> and <tt>Map[URL, Long]</tt>. Since Maps have a monoid, this can be run in parallel, and it will contain a list of who has viewed it and from where. The <tt>Map</tt> has a <tt>Long</tt> since popular tweets can be embedded in millions of websites and so they use a “CountMinSketch” [Note: Reader Sam Bessalah points out that the transcript is wrong when it said “accountment sketch”.] which is an approximate data structure to deal with scale there. The Summingbird layer which the speaker [Sam Ritchie] shows on stage filters events, and generates key-value pairs and emits events.</p>
<p>Twitter advertising is also built on Summingbird. Various campaigns can be built by building a backend using a monoid that expresses the needs, and then the majority of the work is on the UI work in the frontend (where it should be — remember, solve systems problems once is part of the vision).</p>
<p>Source: <a href="http://www.syslog.cl.cam.ac.uk/2013/09/22/liveblogging-cufp--2013/">Transcript of Sam Ritchie’s SummingBird talk at CUFP 2013</a></p>
</blockquote>
<p>See <a href="https://speakerdeck.com/sritchie/summingbird-at-cufp">his CUFP slides on Summingbird</a> for further detail.</p>
<p>Thirdly, you can <em>compose</em> monoids. For instance, you can form the <em>product</em> of two monoids <code class="language-plaintext highlighter-rouge">M1</code> and <code class="language-plaintext highlighter-rouge">M2</code>, which is the
tuple type <code class="language-plaintext highlighter-rouge">(M1, M2)</code>. This product is also a monoid.</p>
<p>Lastly, you can now combine your monoidal data structure with monads (see below) and benefit from all the features that those monads provide.</p>
<div class="note">
At this point you might guess the reason why Ted Dunning was asked whether the <tt>t-digest</tt> data structure he is
working on is associative and can be turned into a semigroup or monoid. One of my two mysteries solved!
</div>
<h2 id="monads">Monads</h2>
<h3 id="what-is-a-monad">What is a monad?</h3>
<div class="note">
Update: A few readers pointed out that this section explains rather what monads are <em>used for</em> than what they
really <em>are</em>. I concur! And I even skip a discussion of Monad laws etc. intentionally because the post is
already quite long, and the focus and motivation of this article (see above) is not an in-depth introduction to monoids
or monads. It's about the questions "Why should I be interested in the first place, and what can I use them for?".
Of course I can understand the need for further details, so I added a list of references and literature to the end of
this article, which you can read at your leisure. Of course if you think that some important piece of information
should be mentioned here directly (or something happens to be plain wrong), please let me know. It's difficult to write
an article about such a topic in a way that can be understood by beginners and at the same time also pleases the
experts.
</div>
<p>A monad is a structure that defines a way to combine <em>functions</em>. It represents computations defined as a <em>sequence</em>
of transformations that turn an original input into a final output, one step at a time. Think of them like function
chaining similar to <code class="language-plaintext highlighter-rouge">y = h(g(f(x)))</code>.</p>
<p>An interesting aspect is that in the case of a monad the <em>type</em> of the value being piped through the function chain may
change along the way. For instance, you may start with an <code class="language-plaintext highlighter-rouge">Int</code> but end up with a <code class="language-plaintext highlighter-rouge">Double</code> or <code class="language-plaintext highlighter-rouge">BloomFilter</code>. This is
different from a monoid, which will always retain the original type because of the <em>closure</em> requirement (see monoid
laws above).</p>
<p>One of the best analogies for monads I found is the following, adapted from
<a href="https://en.wikipedia.org/wiki/Monad_%28functional_programming%29">Wikipedia</a>: You can compare monads to physical
assembly lines, where a conveyor belt (the monad) transports a piece of input material (the data) between functional
units (functions on the data) that transform the piece one step at a time. Think of the skeleton of a car that is
turned into the final car in a sequence of steps. Or of web server log files with raw data that is turned into business
information such as the increase of ad impressions in the EMEA market for this month.</p>
<p><img src="https://www.michael-noll.com/assets/uploads/monad-function-pipeline.png" alt="Monad data processing pipeline" /></p>
<div class="caption">
Figure 3: A monad seen as a data processing pipeline. The monad <tt>M</tt> is used to turn the original input into the
final output one step at a time.
</div>
<p>Sticking with this analogy, a monad enables you to <em>decorate</em> each processing step in the assembly pipeline with
<em>additional context</em> (or an “environment”). For instance, your monad could carry state information that is used by
the functions in the pipeline – this would be the example of a
<a href="https://en.wikibooks.org/wiki/Haskell/Understanding_monads/State">state monad</a>.
Alternatively, your monad could log what is going on before, within, or after a function to a file or database – this
would be the example of an <a href="https://en.wikibooks.org/wiki/Haskell/Understanding_monads/IO">I/O monad</a>.
If you are a game developer, you could use a monad to carry the representation and state of the game environment (such
as the current level), and the functions in the pipeline would model how players can interact with the environment.</p>
<p>Before we look at monads in more detail, let us take a brief detour to <a href="https://github.com/nathanmarz/storm">Storm</a>.
When you are implementing bolts in Storm – i.e. Storm’s version of the “functional units” in a data processing
pipeline – you will come across the <code class="language-plaintext highlighter-rouge">prepare()</code> and <code class="language-plaintext highlighter-rouge">execute()</code> methods
(see the <a href="https://github.com/nathanmarz/storm/wiki/Tutorial">Storm tutorial</a>):</p>
<div class="language-java highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="kd">public</span> <span class="kd">class</span> <span class="nc">TripleBolt</span> <span class="kd">extends</span> <span class="nc">BaseRichBolt</span> <span class="o">{</span>
<span class="kd">private</span> <span class="nc">OutputCollectorBase</span> <span class="n">collector</span><span class="o">;</span>
<span class="c1">// Note how the Storm provides "context" -- a literal context value</span>
<span class="c1">// and a collector value -- to the bolt as the functional unit in</span>
<span class="c1">// the data processing pipeline.</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="nc">Map</span> <span class="n">conf</span><span class="o">,</span> <span class="nc">TopologyContext</span> <span class="n">context</span><span class="o">,</span> <span class="nc">OutputCollectorBase</span> <span class="n">collector</span><span class="o">)</span> <span class="o">{</span>
<span class="k">this</span><span class="o">.</span><span class="na">collector</span> <span class="o">=</span> <span class="n">collector</span><span class="o">;</span>
<span class="o">}</span>
<span class="c1">// This is Storm's version of a monad's `fn` function,</span>
<span class="c1">// which we will discuss in the next section.</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="nc">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="kt">int</span> <span class="n">tripled</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">emit</span><span class="o">(</span><span class="n">input</span><span class="o">,</span> <span class="k">new</span> <span class="nc">Values</span><span class="o">(</span><span class="n">tripled</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="c1">// ...rest omitted...</span>
<span class="o">}</span>
</code></pre></div></div>
<p>Note how Storm provides environmental information and context to the bolt. This is one example where you could point
your finger at the code and say, <em>“This would be a good place to use a monad.”</em> In this specific case I would say it
would be primarily a kind of <em>I/O-monad</em> because the <code class="language-plaintext highlighter-rouge">collector</code> instance allows the bolt write its output to
downstream bolts via network communication.</p>
<h3 id="monads-in-more-detail">Monads in more detail</h3>
<p>Here is one way to capture the concept of a monad in Scala. It is basically the same as the
<a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monad.scala">definition of a monad in Algebird</a>.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Important: What you see here is only part of the contract.</span>
<span class="c1">// The monad, and thus `apply` and `flatMap`, must also adhere to the monad laws.</span>
<span class="k">trait</span> <span class="nc">Monad</span><span class="o">[</span><span class="kt">M</span><span class="o">[</span><span class="k">_</span><span class="o">]]</span> <span class="o">{</span>
<span class="c1">// Also called `unit` (in papers) or `return` (in Haskell).</span>
<span class="k">def</span> <span class="nf">apply</span><span class="o">[</span><span class="kt">T</span><span class="o">](</span><span class="n">v</span><span class="k">:</span> <span class="kt">T</span><span class="o">)</span><span class="k">:</span> <span class="kt">M</span><span class="o">[</span><span class="kt">T</span><span class="o">]</span>
<span class="c1">// Also called `bind` (in papers) or `>>=` (in Haskell).</span>
<span class="k">def</span> <span class="nf">flatMap</span><span class="o">[</span><span class="kt">T</span>, <span class="kt">U</span><span class="o">](</span><span class="n">m</span><span class="k">:</span> <span class="kt">M</span><span class="o">[</span><span class="kt">T</span><span class="o">])(</span><span class="n">fn</span><span class="k">:</span> <span class="o">(</span><span class="kt">T</span><span class="o">)</span> <span class="o">=></span> <span class="n">M</span><span class="o">[</span><span class="kt">U</span><span class="o">])</span><span class="k">:</span> <span class="kt">M</span><span class="o">[</span><span class="kt">U</span><span class="o">]</span>
<span class="o">}</span>
</code></pre></div></div>
<p>Alright, what is going on here?</p>
<p><code class="language-plaintext highlighter-rouge">apply()</code> boxes a <code class="language-plaintext highlighter-rouge">T</code> value into the monad <code class="language-plaintext highlighter-rouge">M[T]</code>. For example, <code class="language-plaintext highlighter-rouge">T</code> is an <code class="language-plaintext highlighter-rouge">Int</code>, the monad <code class="language-plaintext highlighter-rouge">M[T]</code> is a <code class="language-plaintext highlighter-rouge">List[T]</code>.
In other words, it is a good-ol’ constructor for the monad.</p>
<p><code class="language-plaintext highlighter-rouge">flatMap()</code> turns a <code class="language-plaintext highlighter-rouge">T</code> into a potentially different type parameter <code class="language-plaintext highlighter-rouge">U</code> (but it can also be a <code class="language-plaintext highlighter-rouge">T</code> again) that is boxed
into the same type of monad <code class="language-plaintext highlighter-rouge">M</code>, i.e. <code class="language-plaintext highlighter-rouge">M[U]</code>. In plain English, this means that if you have List monad all it will
ever produce for you is another List monad, but the type of elements <em>in the List monad</em> may change. The way this
happens is controlled by the second parameter of <code class="language-plaintext highlighter-rouge">flatMap()</code>, which is a function from <code class="language-plaintext highlighter-rouge">T</code> to <code class="language-plaintext highlighter-rouge">M[U]</code>.</p>
<p>For example, <code class="language-plaintext highlighter-rouge">T</code> is an <code class="language-plaintext highlighter-rouge">Int</code>, <code class="language-plaintext highlighter-rouge">U</code> is a <code class="language-plaintext highlighter-rouge">Double</code>, and <code class="language-plaintext highlighter-rouge">M</code> is a <code class="language-plaintext highlighter-rouge">List</code> monad;
<code class="language-plaintext highlighter-rouge">fn</code> is <code class="language-plaintext highlighter-rouge">(i: Int) => List(i.toDouble / 4, i.toDouble / 2)</code>, i.e. <code class="language-plaintext highlighter-rouge">T -> M[U]</code>.
If you ran this combination over the input <code class="language-plaintext highlighter-rouge">List[Int](1, 2)</code>, you would get the output:
<code class="language-plaintext highlighter-rouge">List[Double](0.25, 0.5, 0.5, 1.0)</code>.</p>
<div class="note">
Note how <tt>flatMap()</tt> provides the boxing <tt>M</tt> instance <tt>m</tt> of the input <tt>T</tt> value to the
function <tt>fn</tt> via currying. This way <tt>fn</tt> may leverage information or functionality embedded in the
monad, including functions beyond the contractually required <tt>flatMap()</tt>. One such example is
<tt>Monad[Some]</tt>, i.e. the
<a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monad.scala">Some monad in Algebird</a>.
The <tt>flatMap()</tt> function of this monad calls <tt>Some#get()</tt>, which is a function of <tt>Some</tt> but not
of <tt>Monad[Some]</tt>. As such a monad is also a kind of adapter or view, similar to the
way we described monoids above. If you still cannot see how similar monoids and monads are, just try squinting harder!
</div>
<p>Similar to the monoid laws we discussed above, monads have their own laws – and these rules are actually very similar
to its monoid brethren! I decided not to discuss monad laws in this post because I feel it is already very long.
I may update the post at a later point though. In the meantime take a look at the following references:</p>
<ul>
<li><a href="http://www.haskell.org/haskellwiki/Monad_laws">Monad laws</a> (in Haskell). Remember <code class="language-plaintext highlighter-rouge">return</code> in Haskell means our
constructor <code class="language-plaintext highlighter-rouge">apply()</code> in Scala, and <code class="language-plaintext highlighter-rouge">>>=</code> in Haskell is our <code class="language-plaintext highlighter-rouge">flatMap()</code>.</li>
<li><a href="http://james-iry.blogspot.ch/2007/09/monads-are-elephants-part-1.html">Monads are elephants</a>, a series of blog posts
by James Iry. In Scala.</li>
</ul>
<p>I hope you will notice their similarities:</p>
<ul>
<li>The identity rules of monads are similar to the identity element <code class="language-plaintext highlighter-rouge">e</code> of monoids.</li>
<li>Both monoids and monads have functions that must be associative.</li>
</ul>
<h3 id="what-are-example-monads">What are example monads?</h3>
<p>At the beginning of the section on monads I already mentioned the <em>state-monad</em> and the <em>I/O-monad</em>.</p>
<p>Well, this may still be a bit vague. Let us look at a more concrete (and maybe simpler) example. Any collection type
is typically a monad. For example, take <code class="language-plaintext highlighter-rouge">List[T]</code>:</p>
<ul>
<li>The constructor of <code class="language-plaintext highlighter-rouge">List[T]</code> acts as <code class="language-plaintext highlighter-rouge">unit</code> as it gives you a <code class="language-plaintext highlighter-rouge">List[T]</code> box for <code class="language-plaintext highlighter-rouge">T</code> instances.</li>
<li><code class="language-plaintext highlighter-rouge">List</code> has an appropriate <code class="language-plaintext highlighter-rouge">flatMap()</code> function – and <code class="language-plaintext highlighter-rouge">map()</code>, which can be built from <code class="language-plaintext highlighter-rouge">flatMap()</code> and the
constructor.</li>
</ul>
<p>Here is the implementation of <code class="language-plaintext highlighter-rouge">Monad[List]</code> <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monad.scala">in Algebird</a>:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">implicit</span> <span class="k">val</span> <span class="nv">list</span><span class="k">:</span> <span class="kt">Monad</span><span class="o">[</span><span class="kt">List</span><span class="o">]</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">Monad</span><span class="o">[</span><span class="kt">List</span><span class="o">]</span> <span class="o">{</span>
<span class="k">def</span> <span class="nf">apply</span><span class="o">[</span><span class="kt">T</span><span class="o">](</span><span class="n">v</span><span class="k">:</span> <span class="kt">T</span><span class="o">)</span> <span class="k">=</span> <span class="nc">List</span><span class="o">(</span><span class="n">v</span><span class="o">);</span>
<span class="k">def</span> <span class="nf">flatMap</span><span class="o">[</span><span class="kt">T</span>,<span class="kt">U</span><span class="o">](</span><span class="n">m</span><span class="k">:</span> <span class="kt">List</span><span class="o">[</span><span class="kt">T</span><span class="o">])(</span><span class="n">fn</span><span class="k">:</span> <span class="o">(</span><span class="kt">T</span><span class="o">)</span> <span class="o">=></span> <span class="nc">List</span><span class="o">[</span><span class="kt">U</span><span class="o">])</span> <span class="k">=</span> <span class="nv">m</span><span class="o">.</span><span class="py">flatMap</span><span class="o">(</span><span class="n">fn</span><span class="o">)</span>
<span class="o">}</span>
</code></pre></div></div>
<p>Here you can see that <code class="language-plaintext highlighter-rouge">Monad[List]</code> is simply a 1:1 adapter for the existing <code class="language-plaintext highlighter-rouge">apply()</code> and <code class="language-plaintext highlighter-rouge">flatMap()</code> functions of
<code class="language-plaintext highlighter-rouge">List</code>. And that’s because <code class="language-plaintext highlighter-rouge">List</code> in Scala already ships with monad “look and feel”.</p>
<p>Before we move on to the next section there is one more interesting facet: A monad can have monoid forms, too.
Algebird, for instance, provides a <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monad.scala">default monoid view for its semigroups and monads</a>:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// This is a Semigroup, for all Monads.</span>
<span class="k">class</span> <span class="nc">MonadSemigroup</span><span class="o">[</span><span class="kt">T</span>,<span class="kt">M</span><span class="o">[</span><span class="k">_</span><span class="o">]](</span><span class="k">implicit</span> <span class="n">monad</span><span class="k">:</span> <span class="kt">Monad</span><span class="o">[</span><span class="kt">M</span><span class="o">],</span> <span class="n">sg</span><span class="k">:</span> <span class="kt">Semigroup</span><span class="o">[</span><span class="kt">T</span><span class="o">])</span>
<span class="k">extends</span> <span class="nc">Semigroup</span><span class="o">[</span><span class="kt">M</span><span class="o">[</span><span class="kt">T</span><span class="o">]]</span> <span class="o">{</span>
<span class="k">import</span> <span class="nn">Monad.operators</span>
<span class="k">def</span> <span class="nf">plus</span><span class="o">(</span><span class="n">l</span><span class="k">:</span> <span class="kt">M</span><span class="o">[</span><span class="kt">T</span><span class="o">],</span> <span class="n">r</span><span class="k">:</span> <span class="kt">M</span><span class="o">[</span><span class="kt">T</span><span class="o">])</span> <span class="k">=</span> <span class="nf">for</span><span class="o">(</span><span class="n">lv</span> <span class="k"><-</span> <span class="n">l</span><span class="o">;</span> <span class="n">rv</span> <span class="k"><-</span> <span class="n">r</span><span class="o">)</span> <span class="k">yield</span> <span class="nv">sg</span><span class="o">.</span><span class="py">plus</span><span class="o">(</span><span class="n">lv</span><span class="o">,</span> <span class="n">rv</span><span class="o">)</span>
<span class="o">}</span>
<span class="c1">// This is a Monoid, for all Monads.</span>
<span class="k">class</span> <span class="nc">MonadMonoid</span><span class="o">[</span><span class="kt">T</span>,<span class="kt">M</span><span class="o">[</span><span class="k">_</span><span class="o">]](</span><span class="k">implicit</span> <span class="n">monad</span><span class="k">:</span> <span class="kt">Monad</span><span class="o">[</span><span class="kt">M</span><span class="o">],</span> <span class="n">mon</span><span class="k">:</span> <span class="kt">Monoid</span><span class="o">[</span><span class="kt">T</span><span class="o">])</span>
<span class="k">extends</span> <span class="nc">MonadSemigroup</span><span class="o">[</span><span class="kt">T</span>,<span class="kt">M</span><span class="o">]</span> <span class="k">with</span> <span class="nc">Monoid</span><span class="o">[</span><span class="kt">M</span><span class="o">[</span><span class="kt">T</span><span class="o">]]</span> <span class="o">{</span>
<span class="k">lazy</span> <span class="k">val</span> <span class="nv">zero</span> <span class="k">=</span> <span class="nf">monad</span><span class="o">(</span><span class="nv">mon</span><span class="o">.</span><span class="py">zero</span><span class="o">)</span>
<span class="o">}</span>
</code></pre></div></div>
<p>Groups, rings, and fields do not have such a default, “automatic” monoid view however. For those algebraic structures
you must check yourself that the group/ring/field laws hold for your monad.</p>
<h3 id="what-can-i-use-a-monad-for--why-should-i-look-for-one">What can I use a monad for? Why should I look for one?</h3>
<p>As we have already seen monads can be thought of as
<a href="http://www.haskell.org/haskellwiki/Monad"><em>composable</em> computation descriptions</a>.
This means you can use them to build powerful data processing pipelines. And these pipelines are not only powerful in
terms of features and functionality, they can also be <em>parallelized</em>, which is one of the reasons why monads are so
attractive in the field of large-scale data processing where your code is run on many cores and on many machines at
the same time.</p>
<div class="note">
Now you might say that almost all we do in coding is to transform one value into another value, and I agree. And this,
I think, is where the idea of the picture "Monads. Monads, everywhere." (see beginning of this article) originates
from. Two of my two mysteries solved, yay!
</div>
<h1 id="algebird">Algebird</h1>
<p>Finally we are getting close to being productive with Algebird. I figure the previous TL;DR section on monoids and
monads was still maybe a bit too long. :-)</p>
<p>If you recall, our original goal at the beginning of this post was to build a data structure <code class="language-plaintext highlighter-rouge">TwitterUser</code> accompanied
with a <code class="language-plaintext highlighter-rouge">Max[TwitterUser]</code> monoid view of it, using Algebird. We wanted to use the two for implementing the analytics of
a simple popularity contest on Twitter:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Let's have a popularity contest on Twitter. The user with the most followers wins!</span>
<span class="k">val</span> <span class="nv">barackobama</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"BarackObama"</span><span class="o">,</span> <span class="mi">40267391</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">katyperry</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"katyperry"</span><span class="o">,</span> <span class="mi">48013573</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">ladygaga</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"ladygaga"</span><span class="o">,</span> <span class="mi">40756470</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">miguno</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"miguno"</span><span class="o">,</span> <span class="mi">731</span><span class="o">)</span> <span class="c1">// I participate, too. Olympic spirit!</span>
<span class="k">val</span> <span class="nv">taylorswift</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"taylorswift13"</span><span class="o">,</span> <span class="mi">37125055</span><span class="o">)</span>
<span class="k">val</span> <span class="nv">winner</span><span class="k">:</span> <span class="kt">Max</span><span class="o">[</span><span class="kt">TwitterUser</span><span class="o">]</span> <span class="k">=</span> <span class="nc">Max</span><span class="o">(</span><span class="n">barackobama</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">katyperry</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">ladygaga</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">miguno</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Max</span><span class="o">(</span><span class="n">taylorswift</span><span class="o">)</span>
<span class="nf">assert</span><span class="o">(</span><span class="nv">winner</span><span class="o">.</span><span class="py">get</span> <span class="o">==</span> <span class="n">katyperry</span><span class="o">)</span>
</code></pre></div></div>
<p>Let’s start!</p>
<h2 id="creating-a-monoid">Creating a monoid</h2>
<h3 id="the-twitteruser-type">The TwitterUser type</h3>
<p>Our first step is to create the data structure <code class="language-plaintext highlighter-rouge">TwitterUser</code> for which we will then create a monoid view.</p>
<p>Because we want to build a <code class="language-plaintext highlighter-rouge">Max</code> monoid for <code class="language-plaintext highlighter-rouge">TwitterUser</code> eventually, we must come up with a way to <em>order</em>
<code class="language-plaintext highlighter-rouge">TwitterUser</code> values. For this we can either use the
<a href="http://www.scala-lang.org/api/current/#scala.math.Ordering">Ordering</a> or the
<a href="http://www.scala-lang.org/api/current/#scala.math.Ordered">Ordered</a> trait in Scala, either way will work.</p>
<p>Let’s say we go down the <code class="language-plaintext highlighter-rouge">Ordered</code> route. Now we must answer a design question: <em>Do we consider the “ordering”
behavior to be a defining feature of <code class="language-plaintext highlighter-rouge">TwitterUser</code> in general, or do we need this behavior only for its
<code class="language-plaintext highlighter-rouge">Max[TwitterUser]</code> monoid view?</em>
If it’s a general feature we would add it to <code class="language-plaintext highlighter-rouge">TwitterUser</code> directly. If it’s only needed for the monoid we can also
decide to add it only there. In our case, we will add the ordering behavior to <code class="language-plaintext highlighter-rouge">TwitterUser</code> directly. I will show
further down below how to implement the other option.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// Small note: To be future-proof we should make `numFollowers` a `Long`,</span>
<span class="c1">// because `Int.MaxValue` (~ 2 billion) is less than the potential number</span>
<span class="c1">// of Twitter users on planet earth. I am happy to let this one slip though.</span>
<span class="k">case</span> <span class="k">class</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="k">val</span> <span class="nv">name</span><span class="k">:</span> <span class="kt">String</span><span class="o">,</span> <span class="k">val</span> <span class="nv">numFollowers</span><span class="k">:</span> <span class="kt">Int</span><span class="o">)</span> <span class="k">extends</span> <span class="nc">Ordered</span><span class="o">[</span><span class="kt">TwitterUser</span><span class="o">]</span> <span class="o">{</span>
<span class="k">def</span> <span class="nf">compare</span><span class="o">(</span><span class="n">that</span><span class="k">:</span> <span class="kt">TwitterUser</span><span class="o">)</span><span class="k">:</span> <span class="kt">Int</span> <span class="o">=</span> <span class="o">{</span>
<span class="k">val</span> <span class="nv">c</span> <span class="k">=</span> <span class="k">this</span><span class="o">.</span><span class="py">numFollowers</span> <span class="o">-</span> <span class="nv">that</span><span class="o">.</span><span class="py">numFollowers</span>
<span class="nf">if</span> <span class="o">(</span><span class="n">c</span> <span class="o">==</span> <span class="mi">0</span><span class="o">)</span> <span class="k">this</span><span class="o">.</span><span class="py">name</span><span class="o">.</span><span class="py">compareTo</span><span class="o">(</span><span class="nv">that</span><span class="o">.</span><span class="py">name</span><span class="o">)</span> <span class="k">else</span> <span class="n">c</span>
<span class="o">}</span>
<span class="o">}</span>
</code></pre></div></div>
<p>The code above means that <code class="language-plaintext highlighter-rouge">TwitterUser</code> supports comparison operations like <code class="language-plaintext highlighter-rouge">>=</code> as defined by the <code class="language-plaintext highlighter-rouge">compare</code> method of
the <code class="language-plaintext highlighter-rouge">Ordered</code> trait.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="n">scala</span><span class="o">></span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"foo"</span><span class="o">,</span> <span class="mi">123</span><span class="o">)</span> <span class="o">></span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"bar"</span><span class="o">,</span> <span class="mi">99999</span><span class="o">)</span>
<span class="n">res5</span><span class="k">:</span> <span class="kt">Boolean</span> <span class="o">=</span> <span class="kc">false</span>
</code></pre></div></div>
<p>In our case this <code class="language-plaintext highlighter-rouge">compare()</code> method is also used as the monoidal binary function of the <code class="language-plaintext highlighter-rouge">Max[TwitterUser]</code> monoid we
will build in the next section. This works because <code class="language-plaintext highlighter-rouge">compare()</code> satisfies all the three axioms described in our section
on monoids above.</p>
<h3 id="the-maxtwitteruser-monoid">The Max[TwitterUser] monoid</h3>
<p>Creating the <code class="language-plaintext highlighter-rouge">Max</code> monoid for <code class="language-plaintext highlighter-rouge">TwitterUser</code> is now very simple because we can leverage a factory method provided by
Algebird’s called <code class="language-plaintext highlighter-rouge">Max.monoid()</code>.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// The "zero" element of the TwitterUser monoid. Traditionally it is</span>
<span class="c1">// also called `mzero` in academic papers. We use `Int.MinValue` here</span>
<span class="c1">// but in practice you would typically constrain `numFollowers` of</span>
<span class="c1">// TwitterUser to be >= 0 anyways, so any negative value such as `-1`</span>
<span class="c1">// would do.</span>
<span class="k">val</span> <span class="nv">zero</span> <span class="k">=</span> <span class="nc">TwitterUser</span><span class="o">(</span><span class="s">"MinUser"</span><span class="o">,</span> <span class="nv">Int</span><span class="o">.</span><span class="py">MinValue</span><span class="o">)</span>
<span class="c1">// Monoid in Algebird is a type class, hence we use implicits</span>
<span class="c1">// to make the monoid available to the rest of the code.</span>
<span class="k">implicit</span> <span class="k">def</span> <span class="nf">twitterUserMonoid</span><span class="k">:</span> <span class="kt">Monoid</span><span class="o">[</span><span class="kt">Max</span><span class="o">[</span><span class="kt">TwitterUser</span><span class="o">]]</span> <span class="k">=</span> <span class="nv">Max</span><span class="o">.</span><span class="py">monoid</span><span class="o">(</span><span class="n">zero</span><span class="o">)</span>
</code></pre></div></div>
<p>That’s it!</p>
<p>Ok, maybe it feels a bit like cheating because the monoid is created behind the scenes by <code class="language-plaintext highlighter-rouge">Max.monoid()</code>.
So what does <code class="language-plaintext highlighter-rouge">Max.monoid()</code> do?</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="cm">/* This is Algebird code, not ours. */</span>
<span class="c1">// Zero should have the property that it <= all T</span>
<span class="k">def</span> <span class="nf">monoid</span><span class="o">[</span><span class="kt">T</span><span class="o">](</span><span class="n">zero</span><span class="k">:</span> <span class="o">=></span> <span class="n">T</span><span class="o">)(</span><span class="k">implicit</span> <span class="n">ord</span><span class="k">:</span> <span class="kt">Ordering</span><span class="o">[</span><span class="kt">T</span><span class="o">])</span><span class="k">:</span> <span class="kt">Monoid</span><span class="o">[</span><span class="kt">Max</span><span class="o">[</span><span class="kt">T</span><span class="o">]]</span> <span class="k">=</span>
<span class="nv">Monoid</span><span class="o">.</span><span class="py">from</span><span class="o">(</span><span class="nc">Max</span><span class="o">(</span><span class="n">zero</span><span class="o">))</span> <span class="o">{</span> <span class="o">(</span><span class="n">l</span><span class="o">,</span><span class="n">r</span><span class="o">)</span> <span class="k">=></span> <span class="nf">if</span><span class="o">(</span><span class="nv">ord</span><span class="o">.</span><span class="py">gteq</span><span class="o">(</span><span class="nv">l</span><span class="o">.</span><span class="py">get</span><span class="o">,</span> <span class="nv">r</span><span class="o">.</span><span class="py">get</span><span class="o">))</span> <span class="n">l</span> <span class="k">else</span> <span class="n">r</span> <span class="o">}</span>
</code></pre></div></div>
<p>Still, it’s pretty straight-forward I would say. Not a lot of magic as long as you know how implicits and type classes
in Scala work.</p>
<div class="note">
Generally, <tt>Max</tt> in Algebird is a semigroup -- not a monoid -- because not all types <tt>T</tt> you could come
up with would have the notion of a zero element when used with <tt>Max</tt>. And the existence of such a zero element
is the one thing that separates a semigroup from a monoid. You see this in Algebird's
<a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/OrderedSemigroup.scala">OrderedSemigroup.scala</a>
where <tt>object Max</tt> defines an <tt>implicit def</tt> semigroup, and only for a few specific types such as
<tt>Int</tt> or <tt>Long</tt> it also defines monoid behavior.. This is because those types have the notion of a zero
element. In our case we have such a zero element, too, hence we can not only support semigroup but also monoid
behavior.
</div>
<p>What would we do if we only wanted to add <code class="language-plaintext highlighter-rouge">compare()</code> to the monoid, but not to the original type?
The Algebird code has examples for this use case. Here is the definition of the <code class="language-plaintext highlighter-rouge">Max[List]</code> monoid, which as you may
notice uses <code class="language-plaintext highlighter-rouge">Ordering</code> and not <code class="language-plaintext highlighter-rouge">Ordered</code> as in our example above. You can ignore that small difference. The key point
is that the <code class="language-plaintext highlighter-rouge">compare()</code> method is defined as part of the <code class="language-plaintext highlighter-rouge">Max[List]</code> monoid instead of being “duct-taped” to <code class="language-plaintext highlighter-rouge">List</code>
directly.</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="k">implicit</span> <span class="k">def</span> <span class="nf">listMonoid</span><span class="o">[</span><span class="kt">T:Ordering</span><span class="o">]</span><span class="k">:</span> <span class="kt">Monoid</span><span class="o">[</span><span class="kt">Max</span><span class="o">[</span><span class="kt">List</span><span class="o">[</span><span class="kt">T</span><span class="o">]]]</span> <span class="k">=</span> <span class="n">monoid</span><span class="o">[</span><span class="kt">List</span><span class="o">[</span><span class="kt">T</span><span class="o">]](</span><span class="nc">Nil</span><span class="o">)(</span><span class="k">new</span> <span class="nc">Ordering</span><span class="o">[</span><span class="kt">List</span><span class="o">[</span><span class="kt">T</span><span class="o">]]</span> <span class="o">{</span>
<span class="nd">@tailrec</span>
<span class="k">final</span> <span class="k">override</span> <span class="k">def</span> <span class="nf">compare</span><span class="o">(</span><span class="n">left</span><span class="k">:</span> <span class="kt">List</span><span class="o">[</span><span class="kt">T</span><span class="o">],</span> <span class="n">right</span><span class="k">:</span> <span class="kt">List</span><span class="o">[</span><span class="kt">T</span><span class="o">])</span><span class="k">:</span> <span class="kt">Int</span> <span class="o">=</span> <span class="o">{</span>
<span class="o">(</span><span class="n">left</span><span class="o">,</span> <span class="n">right</span><span class="o">)</span> <span class="k">match</span> <span class="o">{</span>
<span class="nf">case</span> <span class="o">(</span><span class="nc">Nil</span><span class="o">,</span> <span class="nc">Nil</span><span class="o">)</span> <span class="k">=></span> <span class="mi">0</span>
<span class="nf">case</span> <span class="o">(</span><span class="nc">Nil</span><span class="o">,</span> <span class="k">_</span><span class="o">)</span> <span class="k">=></span> <span class="o">-</span><span class="mi">1</span>
<span class="nf">case</span> <span class="o">(</span><span class="k">_</span><span class="o">,</span> <span class="nc">Nil</span><span class="o">)</span> <span class="k">=></span> <span class="mi">1</span>
<span class="nf">case</span> <span class="o">(</span><span class="n">lh</span><span class="o">::</span><span class="n">lt</span><span class="o">,</span> <span class="n">rh</span><span class="o">::</span><span class="n">rt</span><span class="o">)</span> <span class="k">=></span>
<span class="k">val</span> <span class="nv">c</span> <span class="k">=</span> <span class="nc">Ordering</span><span class="o">[</span><span class="kt">T</span><span class="o">].</span><span class="py">compare</span><span class="o">(</span><span class="n">lh</span><span class="o">,</span> <span class="n">rh</span><span class="o">)</span>
<span class="nf">if</span><span class="o">(</span><span class="n">c</span> <span class="o">==</span> <span class="mi">0</span><span class="o">)</span> <span class="nf">compare</span><span class="o">(</span><span class="n">lt</span><span class="o">,</span> <span class="n">rt</span><span class="o">)</span> <span class="k">else</span> <span class="n">c</span>
<span class="o">}</span>
<span class="o">}</span>
<span class="o">})</span>
</code></pre></div></div>
<h3 id="where-to-go-from-here">Where to go from here?</h3>
<p>Now that we have one monoid view for <code class="language-plaintext highlighter-rouge">TwitterUser</code>, what else can we do? Can we find another monoid form for it?
That’s one of the questions you should ask yourself when working with your own data structures. If you take a look at
the Algebird code, you will notice that many types such as <code class="language-plaintext highlighter-rouge">List</code> will have quite a few algebraic forms.</p>
<p>There is one more thing I want to mention here: You may consider creating an additive monoid for <code class="language-plaintext highlighter-rouge">TwitterUser</code>, i.e.
a monoid that supports a <code class="language-plaintext highlighter-rouge">+</code> like operation. I couldn’t come up with any good example how the result of adding two such
values would make sense (e.g., how could you “add” their usernames in meaningful way?). That being said there is one
case where adding two <code class="language-plaintext highlighter-rouge">TwitterUser</code> values would make sense: to capture the idea that one follows the other, i.e.
to create a relationship (a link) between the two. Keep in mind though that monoids and friends must adhere to the
<em>closure</em> principle – if you start out with a <code class="language-plaintext highlighter-rouge">TwitterUser</code> value and perform monoid operations on it, the end result
must always be another <code class="language-plaintext highlighter-rouge">TwitterUser</code> value. Of course such a relationship can be modeled in code, but you cannot do
this with a <code class="language-plaintext highlighter-rouge">TwitterUser</code> monoid as defined above.</p>
<h2 id="creating-a-monad">Creating a monad?</h2>
<p>By now you should have sufficient understanding of monads and Algebird to implement your own monad. So I leave this
as an exercise for the reader.</p>
<p>A starting point for you is <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monad.scala">Monad.scala</a> in Algebird.</p>
<p>However if you do have a good idea what kind of monad I could showcase here – perhaps something related to Twitter to
match the <code class="language-plaintext highlighter-rouge">TwitterUser</code> monoid example above? – please let me know in the comments.</p>
<h2 id="key-algebraic-structures-in-algebird">Key algebraic structures in Algebird</h2>
<p>The following table is a juxtaposition of a few key algebraic structures, notably those that are implemented in
Algebird. It should help you to navigate the Algebird code base, and also to figure out which algebraic structure
your own data types might support – i.e., <em>“Can I turn my <code class="language-plaintext highlighter-rouge">T</code> into a semigroup, or even a monoid?”</em>.</p>
<table>
<tr>
<th>Algebraic structure</th>
<th>Binary op is associative</th>
<th>Identity (has a zero element)</th>
<th>+ op</th>
<th>- op</th>
<th>* op</th>
<th>/ op</th>
<th>References</th>
</tr>
<tr>
<td>Semigroup</td>
<td>YES</td>
<td>-</td>
<td>YES</td>
<td>-</td>
<td>-</td>
<td>-</td>
<td><a href="https://en.wikipedia.org/wiki/Semigroup">Wikipedia</a>, <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Semigroup.scala">Algebird</a></td>
</tr>
<tr>
<td>Monoid</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>-</td>
<td>-</td>
<td>-</td>
<td><a href="https://en.wikipedia.org/wiki/Monoid">Wikipedia</a>, <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Monoid.scala">Algebird</a></td>
</tr>
<tr>
<td>Group</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>-</td>
<td>-</td>
<td><a href="https://en.wikipedia.org/wiki/Group_%28mathematics%29">Wikipedia</a>, <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Group.scala">Algebird</a></td>
</tr>
<tr>
<td>Ring</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>-</td>
<td><a href="https://en.wikipedia.org/wiki/Ring_%28mathematics%29">Wikipedia</a>, <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Ring.scala">Algebird</a></td>
</tr>
<tr>
<td>Field</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td>YES</td>
<td><a href="https://en.wikipedia.org/wiki/Field_%28mathematics%29">Wikipedia</a>, <a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/field.scala">Algebird</a></td>
</tr>
</table>
<p>Think of <code class="language-plaintext highlighter-rouge">+</code> as the general notion of “adding one thing to another”, same for the other operations. For two
<code class="language-plaintext highlighter-rouge">List[Int]</code>, for instance, <code class="language-plaintext highlighter-rouge">+</code> could be <em>concatenation</em> of the two (instead of, say, trying to add the individual <code class="language-plaintext highlighter-rouge">Int</code>
elements of the lists together). The operators <code class="language-plaintext highlighter-rouge">+</code>, <code class="language-plaintext highlighter-rouge">-</code>, <code class="language-plaintext highlighter-rouge">*</code> and <code class="language-plaintext highlighter-rouge">/</code> are
<a href="https://github.com/twitter/algebird/blob/develop/algebird-core/src/main/scala/com/twitter/algebird/Operators.scala">as defined in Algebird</a>.</p>
<h2 id="a-small-algebird-faq">A small Algebird FAQ</h2>
<h3 id="error-cannot-find-groupmonoid-type-class-for-a-type-t">Error “Cannot find Group/Monoid/… type class for a type T”?</h3>
<p>If you run into this error it means you are trying to use an operation that is not supported by the algebraic structure
you are working with. In this specific example, a <code class="language-plaintext highlighter-rouge">Set()</code> in Algebird has a monoid form and thus supports an
addition-like operation <code class="language-plaintext highlighter-rouge">+</code> but not a multiplication-like operation <code class="language-plaintext highlighter-rouge">*</code>.</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>scala> Set(1,2,3) * Set(2,3,4)
<console>:2: error: Cannot find Ring type class for scala.collection.immutable.Set[Int]
Set(1,2,3) * Set(2,3,4)
</code></pre></div></div>
<h3 id="combine-different-monoids">Combine different monoids?</h3>
<p>In theory you <em>can</em> combine different monoids such as <code class="language-plaintext highlighter-rouge">Max[Int]</code> and <code class="language-plaintext highlighter-rouge">Min[Int]</code> and form their product, but there must
exist an appropriate algebraic structure for that product. Right now, for instance, the following code will not work
in Algebird because it does not ship with a algebraic structure for <code class="language-plaintext highlighter-rouge">(Max[Int], Min[Int])</code>:</p>
<div class="language-scala highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="n">scala</span><span class="o">></span> <span class="nc">Max</span><span class="o">(</span><span class="mi">3</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Min</span><span class="o">(</span><span class="mi">4</span><span class="o">)</span>
<span class="o"><</span><span class="n">console</span><span class="k">>:</span><span class="mi">14</span><span class="k">:</span> <span class="kt">error:</span> <span class="kt">Cannot</span> <span class="kt">find</span> <span class="kt">Semigroup</span> <span class="k">type</span> <span class="kt">class</span> <span class="kt">for</span> <span class="kt">Product</span> <span class="kt">with</span> <span class="kt">Serializable</span>
<span class="nc">Max</span><span class="o">(</span><span class="mi">3</span><span class="o">)</span> <span class="o">+</span> <span class="nc">Min</span><span class="o">(</span><span class="mi">4</span><span class="o">)</span>
</code></pre></div></div>
<h1 id="are-monads-really-everywhere">Are monads really everywhere?</h1>
<p>One thing that I have not yet investigated in further detail is how using monads compares to other patterns of
abstraction. For instance, you can use <a href="http://www.clojure.net/2012/02/02/Monads-in-Clojure/">monads in Clojure</a>
(the author Jim Duey actually wrote a
<a href="http://www.clojure.net/tags.html#monads-ref">whole series of blog posts covering monads</a>), too, but in a quick initial
search I observed that Clojure developers apparently use different constructs to achieve similar effects.</p>
<p>If you have some insights to share here, please feel free to reply to this post!</p>
<h1 id="summary">Summary</h1>
<p>I hope this post contributes a little bit to the understanding of the rather abstract concepts of monoids and monads,
and how you can put them to good practical use via tools such as <a href="https://github.com/twitter/algebird">Algebird</a>,
<a href="https://github.com/twitter/scalding">Scalding</a> and <a href="https://github.com/twitter/summingbird">SummingBird</a>.</p>
<p>One of my lessons learned was that working with monoids and monads is a nice opportunity to read up on more formal
concepts (category theory), and at the same time realize how they can be put to practical use in engineering, notably
when doing large-scala data analytics.</p>
<p>On my side I want to thank the Twitter engineering team (<a href="https://twitter.com/TwitterEng">@TwitterEng</a>) not only for
making those tools available to the open source community, but also for sparking my interest in the practical
application of algebraic structures and category theory in general. Same shout-out for all the various people who
wrote blog posts on the topic, or who shared their insights on places such as StackOverflow (see the reference section
at the end of this article for a few of them). As I said there was a lot of new information to swallow – and in a
short period of time – but the quest was worth it.</p>
<p>Many thanks! <em>–Michael</em></p>
<h1 id="references">References</h1>
<h2 id="monads-and-monoids">Monads and monoids</h2>
<p>I tried to categorize the references below into “easy” and “advanced” reads. Of course this is highly subjective, and
your mileage may vary.</p>
<p>Easy reads:</p>
<ul>
<li><a href="http://www.manning.com/bjarnason/">Functional Programming in Scala</a> by P. Chiusano and R. Bjarnason, published by
Manning. Includes chapters on monoids and monads, and how to implement them in Scala.</li>
<li><a href="http://www.codecommit.com/blog/ruby/monads-are-not-metaphors">Monads are not metaphors</a>, by Daniel Spiewak.</li>
<li><a href="http://stackoverflow.com/questions/3870088">A monad is just a monoid in the category of endofunctors, what’s the problem?</a>,
question on StackOverflow. If you are just starting out with monads etc. I’d recommend to read the
<a href="http://stackoverflow.com/a/7829607/1743580">second answer</a> first.</li>
<li><a href="http://james-iry.blogspot.ch/2007/09/monads-are-elephants-part-1.html">Monads are elephants</a>, a series of blog posts
by James Iry. In Scala.</li>
<li><a href="http://adit.io/posts/2013-04-17-functors,_applicatives,_and_monads_in_pictures.html">Functors, Applicatives, And Monads In Pictures</a>,
by Aditya Bhargava.</li>
<li><a href="http://www.haskell.org/haskellwiki/Monad_laws">Monad laws</a> (in Haskell). Remember <code class="language-plaintext highlighter-rouge">return</code> in Haskell means our
constructor <code class="language-plaintext highlighter-rouge">apply()</code> in Scala, and <code class="language-plaintext highlighter-rouge">>>=</code> in Haskell is our <code class="language-plaintext highlighter-rouge">flatMap()</code>.</li>
<li>Wikipedia articles on algebraic structures: I found that selective reading of those did help my understanding (I did
not try to understand all the sections in those articles). Notably, I liked the juxtaposition of semigroups, monoid,
groups, rings, etc. which highlighted their similarities and differences. Later on I discovered that the Algebird
code is structured similarly, so if you can tell a semigroup from a monoid you will have an easier time navigating
the code.
<ul>
<li><a href="https://en.wikipedia.org/wiki/Semigroup">Semigroup</a></li>
<li><a href="https://en.wikipedia.org/wiki/Monoid">Monoid</a></li>
<li><a href="https://en.wikipedia.org/wiki/Group_%28mathematics%29">Group</a></li>
<li><a href="https://en.wikipedia.org/wiki/Ring_%28mathematics%29">Ring</a></li>
<li><a href="https://en.wikipedia.org/wiki/Monad_%28functional_programming%29">Monad</a></li>
</ul>
</li>
</ul>
<p>Advanced reads:</p>
<ul>
<li><a href="http://www.stephendiehl.com/posts/monads.html">Monads Made Difficult</a>, by Stephen Diel. In Haskell.</li>
<li><a href="http://www.clojure.net/2012/02/02/Monads-in-Clojure/">Monads in Clojure</a>, by Jim Duey. In Clojure. Jim actually
wrote a <a href="http://www.clojure.net/tags.html#monads-ref">whole series of blog posts covering monads</a>.</li>
<li><a href="http://learnyouahaskell.com/functors-applicative-functors-and-monoids">Functors, Applicative Functors and Monoids</a>,
a chapter in <a href="http://learnyouahaskell.com">Learn You a Haskell</a>.</li>
</ul>
<h2 id="summingbird">SummingBird</h2>
<ul>
<li><a href="https://speakerdeck.com/sritchie/summingbird-at-cufp">SummingBird at CUFP 2013</a>, slides by Sam Ritchie (former
Twitter engineer)</li>
</ul>
<h2 id="category-theory">Category theory</h2>
<p>Speaking from my own experience I would say you do not need to understand the full details of category theory. The
links above should contain all the information you need to gain enough understanding of monoids, monads and such that
you can be productive in a short period of time. However I have used the references below to fill gaps that remained
after reading through the other sources above, and I remember I did jump back and forth between the academic references
below and the more hands-on resources above.</p>
<ul>
<li><a href="http://www.amazon.com/dp/0521283043">An Introduction to Category Theory</a>, by Harold Simmons. As a novice to category
theory I preferred this text over <em>Category Theory for Computing Science</em> (see below). Unlike the latter though the
book of Simmons is not available for free.</li>
<li><em>Category Theory for Computing Science</em>, by Michael Barr and Charles Wells, available as a
<a href="http://www.math.mcgill.ca/triples/Barr-Wells-ctcs.pdf">free PDF</a>. This seems to be a seminal work on category
theory and worth the read if you are interested in the mathematical foundation of the theory in about 400 pages.</li>
</ul>Have you ever asked yourself what monoids and monads are, and particularly why they seem to be so attractive in the field of large-scale data processing? Twitter recently open-sourced Algebird, which provides you with a JVM library to work with such algebraic data structures. Algebird is already being used in Big Data tools such as Scalding and SummingBird, which means you can use Algebird as a mechanism to plug your own data structures – e.g. Bloom filters, HyperLogLog – directly into large-scale data processing platforms such as Hadoop and Storm. In this post I will show you how to get started with Algebird, introduce you to monoids and monads, and address the question why you should get interested in those in the first place.Sending Metrics from Storm to Graphite2013-11-06T16:00:00+01:002013-11-06T16:00:00+01:00https://www.michael-noll.com/blog/2013/11/06/sending-metrics-from-storm-to-graphite<p>So you got your first <a href="/tutorials/running-multi-node-storm-cluster/">distributed Storm cluster installed</a> and have
your first topologies up and running. Great! Now you want to integrate your Storm applications with your monitoring
systems and begin tracking application-level metrics from your topologies. In this article I show you how to
integrate Storm with the popular Graphite monitoring system. This, combined with the Storm UI, will provide you with
actionable information to
<a href="/blog/2013/06/21/understanding-storm-internal-message-buffers/">tune the performance of your topologies</a> and also help
you to track key business as well as technical metrics.</p>
<!-- more -->
<ul id="markdown-toc">
<li><a href="#background-what-is-graphite" id="markdown-toc-background-what-is-graphite">Background: What is Graphite?</a></li>
<li><a href="#what-we-want-to-do" id="markdown-toc-what-we-want-to-do">What we want to do</a> <ul>
<li><a href="#spatial-granularity-of-metrics" id="markdown-toc-spatial-granularity-of-metrics">Spatial granularity of metrics</a></li>
<li><a href="#temporal-granularity-of-metrics" id="markdown-toc-temporal-granularity-of-metrics">Temporal granularity of metrics</a></li>
<li><a href="#assumptions" id="markdown-toc-assumptions">Assumptions</a></li>
</ul>
</li>
<li><a href="#desired-outcome-graphs-and-dashboards" id="markdown-toc-desired-outcome-graphs-and-dashboards">Desired outcome: graphs and dashboards</a></li>
<li><a href="#versions" id="markdown-toc-versions">Versions</a></li>
<li><a href="#a-graphite-primer" id="markdown-toc-a-graphite-primer">A Graphite primer</a> <ul>
<li><a href="#understanding-how-graphite-handles-incoming-data" id="markdown-toc-understanding-how-graphite-handles-incoming-data">Understanding how Graphite handles incoming data</a></li>
<li><a href="#implications-of-storms-execution-model" id="markdown-toc-implications-of-storms-execution-model">Implications of Storm’s execution model</a></li>
</ul>
</li>
<li><a href="#high-level-approach" id="markdown-toc-high-level-approach">High-level approach</a> <ul>
<li><a href="#overview-of-the-approach-described-in-this-post" id="markdown-toc-overview-of-the-approach-described-in-this-post">Overview of the approach described in this post</a></li>
<li><a href="#other-approaches-not-used" id="markdown-toc-other-approaches-not-used">Other approaches (not used)</a></li>
</ul>
</li>
<li><a href="#integrating-storm-with-graphite" id="markdown-toc-integrating-storm-with-graphite">Integrating Storm with Graphite</a> <ul>
<li><a href="#configuring-graphite" id="markdown-toc-configuring-graphite">Configuring Graphite</a> <ul>
<li><a href="#carbon-cache-configuration" id="markdown-toc-carbon-cache-configuration">carbon-cache configuration</a></li>
<li><a href="#carbon-aggregator-configuration" id="markdown-toc-carbon-aggregator-configuration">carbon-aggregator configuration</a></li>
<li><a href="#other-important-graphite-settings" id="markdown-toc-other-important-graphite-settings">Other important Graphite settings</a></li>
</ul>
</li>
<li><a href="#configuring-your-storm-code" id="markdown-toc-configuring-your-storm-code">Configuring your Storm code</a> <ul>
<li><a href="#add-the-metrics-library-to-your-storm-code-project" id="markdown-toc-add-the-metrics-library-to-your-storm-code-project">Add the Metrics library to your Storm code project</a></li>
<li><a href="#sending-metrics-from-a-storm-bolt-to-graphite" id="markdown-toc-sending-metrics-from-a-storm-bolt-to-graphite">Sending metrics from a Storm bolt to Graphite</a></li>
</ul>
</li>
</ul>
</li>
<li><a href="#summary" id="markdown-toc-summary">Summary</a></li>
<li><a href="#appendix" id="markdown-toc-appendix">Appendix</a> <ul>
<li><a href="#where-to-go-from-here" id="markdown-toc-where-to-go-from-here">Where to go from here</a></li>
<li><a href="#caveat-storm-samples-metrics-for-the-storm-ui" id="markdown-toc-caveat-storm-samples-metrics-for-the-storm-ui">Caveat: Storm samples metrics for the Storm UI</a></li>
</ul>
</li>
</ul>
<div class="note">
<strong>Update March 13, 2015</strong>: We have open sourced
<a href="https://github.com/verisign/storm-graphite">storm-graphite</a>, an Storm IMetricsConsumer implementation that
forwards Storm's <em>built-in metrics</em> to a Graphite server for real-time graphing, visualization, and operational
dashboards. These built-in metrics greatly augment the application-level metrics that you can send from your Storm
topologies to Graphite (sending application metrics is described in this article). The built-in metrics include
execution count and latency of your bolts, Java heap space usage and garbage collection statistics, and much more.
So if you are interested in even better metrics and deeper insights into your Storm cluster, I'd strongly recommend to
take a look at
<a href="https://github.com/verisign/storm-graphite">storm-graphite</a>. We also describe how to configure Graphite
and Grafana, a dashboard for Graphite, to make use of the built-in metrics provided by storm-graphite.
</div>
<h1 id="background-what-is-graphite">Background: What is Graphite?</h1>
<p>Quoting from <a href="http://graphite.readthedocs.org/en/latest/overview.html">Graphite’s documentation</a>, Graphite does two
things:</p>
<ol>
<li>Store numeric time-series data</li>
<li>Generate and render graphs of this data on demand</li>
</ol>
<p>What Graphite does not do is collect the actual input data for you, i.e. your system or application metrics. The
purpose of this blog post is to show how you can do this for your Storm applications.</p>
<div class="note">
Note: The Graphite project is currently undergoing significant changes. The project has been moved to GitHub and split
into individual components. Also, the next version of Graphite will include for Ceres, which is a distributable
time-series database, and a major refactor of its Carbon daemon. If that draws your interest then you can
<a href="http://graphite.wikidot.com/">read about the upcoming changes in further detail</a>. I mention this just for
completeness -- it should not deter you from jumping on the Graphite bandwagon.
</div>
<h1 id="what-we-want-to-do">What we want to do</h1>
<h2 id="spatial-granularity-of-metrics">Spatial granularity of metrics</h2>
<p>For the context of this post we want to use Graphite to track the number of received tuples of an example bolt
<em>per node</em> in the Storm cluster. This allows us, say, to pinpoint a potential topology bottleneck to specific machines
in the Storm cluster – and this is particularly powerful if we already track system metrics (CPU load, memory usage,
network traffic and such) in Graphite because then you can correlate system and application level metrics.</p>
<p>Keep in mind that in Storm multiple instances of a bolt may run on a given node, and its instances may also run on many
different nodes. Our challenge will be to configure Storm and Graphite in a way that we are able to correctly collect
and aggregate all individual values reported by those many instances of the bolt. Also, the total value of these
per-host tuple counts should ideally match the bolt’s <code class="language-plaintext highlighter-rouge">Executed</code> value – which means the number of executed tuples of
a bolt (i.e. across all instances of the bolt in a topology) – in the Storm UI.</p>
<p>We will add Graphite support to our Java-based Storm topology by using Coda Hale/Yammer’s
<a href="http://metrics.codahale.com/">Metrics library for Java</a>, which directly supports
<a href="http://metrics.codahale.com/manual/graphite/">reporting metrics to Graphite</a>.</p>
<p>We will track the number of received tuples of our example bolt through the following metrics, where <em>HOSTNAME</em> is a
placeholder for the hostname of a particular Storm node (e.g. <code class="language-plaintext highlighter-rouge">storm-node01</code>):</p>
<ul>
<li><code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.HOSTNAME.tuples.received.count</code></li>
<li><code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.HOSTNAME.tuples.received.m1_rate</code> – 1-minute rate</li>
<li><code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.HOSTNAME.tuples.received.m5_rate</code> – 5-minute rate</li>
<li><code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.HOSTNAME.tuples.received.m15_rate</code> – 15-minute rate</li>
<li><code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.HOSTNAME.tuples.received.mean_rate</code> – average rate/sec</li>
</ul>
<p>Here, the prefix of the metric namespace <code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.HOSTNAME.tuples.received</code> is defined by us.
Splitting up this “high-level” metric into a <code class="language-plaintext highlighter-rouge">count</code> metric and four rate metrics – <code class="language-plaintext highlighter-rouge">m{1,5,15}_rate</code> and <code class="language-plaintext highlighter-rouge">mean_rate</code> –
is automatically done by the Metrics Java library.</p>
<h2 id="temporal-granularity-of-metrics">Temporal granularity of metrics</h2>
<p>Because Storm is a real-time analytics platform we want to use a shorter time window for metrics updates than Graphite’s
default, which is one minute. In our case we will report metrics data every 10 seconds (the finest granularity that
Graphite supports is one second).</p>
<h2 id="assumptions">Assumptions</h2>
<ul>
<li>We are using a single Graphite server called <code class="language-plaintext highlighter-rouge">your.graphite.server.com</code>.</li>
<li>The <code class="language-plaintext highlighter-rouge">carbon-cache</code> and <code class="language-plaintext highlighter-rouge">carbon-aggregator</code> daemons of Graphite are both running on the Graphite server machine, i.e.
<code class="language-plaintext highlighter-rouge">carbon-aggregator</code> will send its updates to the <code class="language-plaintext highlighter-rouge">carbon-cache</code> daemon running at <code class="language-plaintext highlighter-rouge">127.0.0.1</code>. Also, our Storm
topology will send all its metrics to this Graphite server.</li>
</ul>
<p>Thankfully the specifics of the Storm cluster such as hostnames of nodes do not matter. So the approach described here
should work nicely with your existing Storm cluster.</p>
<h1 id="desired-outcome-graphs-and-dashboards">Desired outcome: graphs and dashboards</h1>
<p>The desired end result are graphs and dashboards similar to the following Graphite screenshot:</p>
<p><img src="https://www.michael-noll.com/assets/uploads/graphitedemo-storm-dashboard.png" alt="Example graph in Graphite that displays number of received tuples" /></p>
<div class="caption">
Example graph in Graphite that displays the number of received tuples. The brown line is the aggregate of all per-host
tuple counts of this 4-node Storm cluster and computed via Graphite's
<a href="http://graphite.readthedocs.org/en/latest/functions.html#graphite.render.functions.sumSeries">sumSeries()</a>
function. Note that only 3 of the 4 nodes are actually running instances of the bolt, hence you only see 3+1 lines in
the graph.
</div>
<h1 id="versions">Versions</h1>
<p>The instructions in this article have been tested on RHEL/CentOS 6 with the following software versions:</p>
<ul>
<li><a href="http://storm-project.net/">Storm</a> 0.9.0-rc2</li>
<li><a href="http://graphite.wikidot.com/">Graphite</a> 0.9.12 (stock version available in EPEL for RHEL6)</li>
<li><a href="http://metrics.codahale.com/">Metrics</a> 3.0.1</li>
<li>Oracle JDK 6</li>
</ul>
<p>Note that I will not cover the installation of Storm or Graphite in this post.</p>
<div class="note">
Heads up: I am currently working on open sourcing an automated deployment tool called Wirbelsturm that you can use to
install Storm clusters and Graphite servers (and other Big Data related software packages) from scratch. Wirbelsturm
is based on the popular deployment tools <a href="http://puppetlabs.com/">Puppet</a> and
<a href="http://www.vagrantup.com/">Vagrant</a>. Please stay tuned!
</div>
<h1 id="a-graphite-primer">A Graphite primer</h1>
<h2 id="understanding-how-graphite-handles-incoming-data">Understanding how Graphite handles incoming data</h2>
<p>One pitfall for Graphite beginners is the default behavior of Graphite to discard all but the last update message
received during a given time slot (the default size of a time slot for metrics in Graphite is 60 seconds). For example,
if we are sending the metric values <code class="language-plaintext highlighter-rouge">5</code> and <code class="language-plaintext highlighter-rouge">4</code> during the same time slot then Graphite will first store a value of <code class="language-plaintext highlighter-rouge">5</code>,
and as soon as the value <code class="language-plaintext highlighter-rouge">4</code> arrives it will overwrite the stored value from <code class="language-plaintext highlighter-rouge">5</code> to <code class="language-plaintext highlighter-rouge">4</code> (but not sum it up to <code class="language-plaintext highlighter-rouge">9</code>).</p>
<p>The following diagram shows what happens if Graphite receives multiple updates during the same time slot when we are NOT
using an aggregator such as <a href="http://graphite.readthedocs.org/en/latest/carbon-daemons.html">carbon-aggregator</a> or
<a href="https://github.com/etsy/statsd">statsd</a> in between. In this example we use a time slot of 10 seconds for the metric.
Note again that in this scenario you might see, for instance, “flapping” values for the second time slot (window of
seconds 10 to 20) depending on <em>when</em> you would query Graphite: If you queried Graphite at second 15 for the 10-20 time
slot, you would receive a return value of <code class="language-plaintext highlighter-rouge">3</code>, and if you queried only a few seconds later you would start receiving the
final value of <code class="language-plaintext highlighter-rouge">7</code> ( the latter of which would then never change anymore).</p>
<p><img src="https://www.michael-noll.com/assets/uploads/Graphite-update-behavior-01.png" alt="Example Graphite behavior without carbon-aggregator or statsd" /></p>
<p>In most situations losing all but the last update of a given time slot is not what you want. The next diagram shows how
aggregators solve the “only the last update counts” problem. A nice property of aggregators is that they are
transparent to the client who can continue to send updates as soon as it sees fit – the aggregators will ensure that
Graphite will only see a single, merged update message for the time slot.</p>
<p><img src="https://www.michael-noll.com/assets/uploads/Graphite-update-behavior-02.png" alt="Example Graphite behavior with carbon-aggregator or statsd" /></p>
<h2 id="implications-of-storms-execution-model">Implications of Storm’s execution model</h2>
<p>In the case of Storm you implement a bolt (or spout) as a single class, e.g. by extending <code class="language-plaintext highlighter-rouge">BaseBasicBolt</code>. So following
the <a href="http://metrics.codahale.com/manual/">User Manual</a> of the Metrics library seems to be a straight-forward way to add
Graphite support to your Storm bolts. However you must be aware of how Storm will actually execute your topology
behind the scenes – see my earlier post on
<a href="/blog/2012/10/16/understanding-the-parallelism-of-a-storm-topology/">Understanding the Parallelism of a Storm Topology</a>:</p>
<ol>
<li>In Storm each bolt typically runs in the form of many bolt instances in a single worker process, and thus you have
many bolt instances in a single JVM.</li>
<li>In Storm there are typically many such workers (and thus JVMs) per machine, so you end up with many instances of the
same bolt running across many workers/JVMs on a particular machine.</li>
<li>On top of that a bolt’s instances will also be spread across many different machines in the Storm cluster, so in
total you will typically have many bolt instances running in many JVMs across many Storm nodes.</li>
</ol>
<p>Our challenge to integrate Storm with Graphite can thus be stated as: How can we ensure that we are reporting metrics
from our Storm topology to Graphite in such way that a) we are <em>counting</em> tuples correctly across all bolt instances,
and b) the many metric update messages are not canceling each other out? In other words, how can we keep Storm’s
highly distributed nature in check and make it play nice with Graphite?</p>
<h1 id="high-level-approach">High-level approach</h1>
<h2 id="overview-of-the-approach-described-in-this-post">Overview of the approach described in this post</h2>
<p>Here is an overview of the approach we will be using:</p>
<ul>
<li><em>Each instance</em> of our example Storm bolt gets its own (Java) instance of
<a href="http://metrics.codahale.com/manual/core/">Meter</a>. This ensures that each bolt instance tracks its count of received
tuples separately from any other bolt instance.</li>
<li>Also, each bolt instance will get its own instance of <a href="http://metrics.codahale.com/manual/graphite/">GraphiteReporter</a>
to ensures that each bolt instance sends only a single metrics update every 10 seconds, which is the desired temporal
granularity for our monitoring setup.</li>
<li>All bolt instances on a given Storm node report their metrics under the node’s <em>hostname</em>. For instance, bolt
instances on the machine <code class="language-plaintext highlighter-rouge">storm-node01.example.com</code> will report their metrics under the namespace
<code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.storm-node01.tuples.received.*</code>.</li>
<li>Metrics are being sent to a <code class="language-plaintext highlighter-rouge">carbon-aggregator</code> instance running at <code class="language-plaintext highlighter-rouge">your.graphite.server.com:2023/tcp</code>. The
<code class="language-plaintext highlighter-rouge">carbon-aggregator</code> ensures that all the individual metrics updates (from bolt instances) of a particular Storm node
are aggregated into a single, per-host metric update. These per-host metric updates are then forwarded to the
<code class="language-plaintext highlighter-rouge">carbon-cache</code> instance, which will store the metric data in the corresponding Whisper database files.</li>
</ul>
<h2 id="other-approaches-not-used">Other approaches (not used)</h2>
<p>Another strategy is to install an aggregator intermediary (such as <a href="https://github.com/etsy/statsd">statsd</a>) on each
machine in the Storm cluster. Instances of a bolt on the same machine would be sending their individual updates to this
per-host aggregator daemon, which in turn would send a single, per-host update message to Graphite. I am sure this
approach would have worked but I decided to not go down this path. It would have increased the deployment complexity
because now we’d have one more software package to understand, support and manage per machine.</p>
<p>The final setup described in this post achieves what we want by using <code class="language-plaintext highlighter-rouge">GraphiteReporter</code> in our Storm code in a way
that is compatible with Graphite’s built-in daemons without needing any additional software such as <code class="language-plaintext highlighter-rouge">statsd</code>.</p>
<p>On a completely different note, Storm 0.9 now also comes with its own metrics system, which I do not cover here.
This new metrics feature of Storm allows you to collect arbitrarily custom metrics over fixed time windows. Those
metrics are exported to a metrics stream that you can consume by implementing
<a href="https://github.com/nathanmarz/storm/blob/master/storm-core/src/jvm/backtype/storm/metric/api/IMetricsConsumer.java">IMetricsConsumer</a>
and configured with
<a href="https://github.com/nathanmarz/storm/blob/master/storm-core/src/jvm/backtype/storm/Config.java">Config</a> – see the
various <code class="language-plaintext highlighter-rouge">*_METRICS_*</code> settings. Then you need to use <code class="language-plaintext highlighter-rouge">TopologyContext#registerMetric()</code> to register new metrics.</p>
<h1 id="integrating-storm-with-graphite">Integrating Storm with Graphite</h1>
<h2 id="configuring-graphite">Configuring Graphite</h2>
<p>I will only cover the key settings of Graphite for the context of this article, which are the settings related to
<code class="language-plaintext highlighter-rouge">carbon-cache</code> and <code class="language-plaintext highlighter-rouge">carbon-aggregator</code>. <strong>Those settings must match the settings in your Storm code.</strong> Matching
settings between Storm and Graphite is critical – if they don’t you will end up with junk metric data.</p>
<h3 id="carbon-cache-configuration">carbon-cache configuration</h3>
<p>First we must add a <code class="language-plaintext highlighter-rouge">[production_apps]</code> section (the name itself is not relevant, it should only be descriptive) to
<code class="language-plaintext highlighter-rouge">/etc/carbon/storage-schemas.conf</code>. This controls at which granularity Graphite will store incoming metrics that we are
sending from our Storm topology. Notably these storage schema settings control:</p>
<ul>
<li>The minimum temporal granularity for the “raw” incoming metric updates of a given metric namespace: In our case, for
instance, we want Graphite to track metrics at a raw granularity of 10 seconds for the first two days. We configure
this via <code class="language-plaintext highlighter-rouge">10s:2d</code>. This minimum granularity (10 seconds) <strong>must match</strong> the report interval we use in our Storm code.</li>
<li>How Graphite aggregates older metric values that have already been stored in its Whisper database files:
In our case we tell Graphite to aggregate any values older than two days into 5-minute buckets that we want to keep
for one year, hence <code class="language-plaintext highlighter-rouge">5m:1y</code>. This setting (5 minutes) is independent from our Storm code.</li>
</ul>
<div class="warning">
Caution: Graphite knows two different kinds of aggregation. First, the aggregation of metrics data that is already
stored in its Whisper database files; this aggregation is performed to save disk storage space and performed on aging
data. Second, the real-time aggregation of incoming metrics performed by <tt>carbon-aggregator</tt>; this aggregation
happens for newly received data as it is flying in over the network, i.e. before that data even hits the Whisper
database files. Do not confuse these two aggregations!
</div>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># File: /etc/carbon/storage-schemas.conf
# Schema definitions for whisper files. Entries are scanned in order, and first match wins.
[carbon]
pattern = ^carbon\.
retentions = 60:90d
[production_apps]
pattern = ^production\.apps\.
retentions = 10s:2d,5m:1y
[default_1min_for_1day]
pattern = .*
retentions = 60s:1d
</code></pre></div></div>
<p>Next we must tell Graphite which aggregation method – e.g. <code class="language-plaintext highlighter-rouge">sum</code> or <code class="language-plaintext highlighter-rouge">average</code> – it should use to perform storage
aggregation of our metrics. For count-type metrics, for instance, we want to use <code class="language-plaintext highlighter-rouge">sum</code> and for rate-type metrics we
want to use <code class="language-plaintext highlighter-rouge">average</code>. By adding the following lines to <code class="language-plaintext highlighter-rouge">/etc/carbon/storage-aggregation.conf</code> we ensure that Graphite
correctly aggregates the default metrics sent by Metrics’ GraphiteReporter – <code class="language-plaintext highlighter-rouge">count</code>, <code class="language-plaintext highlighter-rouge">m1_rate</code>, <code class="language-plaintext highlighter-rouge">m5_rate</code>, <code class="language-plaintext highlighter-rouge">m15_rate</code>
and <code class="language-plaintext highlighter-rouge">mean_rate</code> – once two days have passed.</p>
<div class="note">
Note: The <tt>[min]</tt> and <tt>[max]</tt> sections are actually not used by the setup described in this article but I
decided to include them anyways to show the difference to the other settings. Also, your production Graphite setup may
actually need to use such settings, too.
</div>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># File: /etc/carbon/storage-aggregation.conf
[min]
pattern = \.min$
xFilesFactor = 0.1
aggregationMethod = min
[max]
pattern = \.max$
xFilesFactor = 0.1
aggregationMethod = max
[sum]
pattern = \.count$
xFilesFactor = 0
aggregationMethod = sum
[m1_rate]
pattern = \.m1_rate$
xFilesFactor = 0
aggregationMethod = average
[m5_rate]
pattern = \.m5_rate$
xFilesFactor = 0
aggregationMethod = average
[m15_rate]
pattern = \.m15_rate$
xFilesFactor = 0
aggregationMethod = average
[default_average]
pattern = .*
xFilesFactor = 0.3
aggregationMethod = average
</code></pre></div></div>
<p>Lastly, make sure that the <code class="language-plaintext highlighter-rouge">carbon-cache</code> daemon is actually enabled in your <code class="language-plaintext highlighter-rouge">/etc/carbon/carbon.conf</code> and configured to
receive incoming data on its <code class="language-plaintext highlighter-rouge">LINE_RECEIVER_PORT</code> at <code class="language-plaintext highlighter-rouge">2003/tcp</code> and also (!) on its <code class="language-plaintext highlighter-rouge">PICKLE_RECEIVER_PORT</code> at
<code class="language-plaintext highlighter-rouge">2004/tcp</code>. The latter port is used by <code class="language-plaintext highlighter-rouge">carbon-aggregator</code>, which we will configure in the next section.</p>
<p>Example configuration snippet:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># File: /etc/carbon/carbon.conf
# ...snipp...
[cache]
LINE_RECEIVER_INTERFACE = 0.0.0.0
LINE_RECEIVER_PORT = 2003
PICKLE_RECEIVER_INTERFACE = 0.0.0.0
PICKLE_RECEIVER_PORT = 2004
# ...snipp...
</code></pre></div></div>
<p>Don’t forget to restart <code class="language-plaintext highlighter-rouge">carbon-cache</code> after changing its configuration:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ sudo service carbon-cache restart
</code></pre></div></div>
<h3 id="carbon-aggregator-configuration">carbon-aggregator configuration</h3>
<p>The last Graphite configuration we must perform is to ensure that we can pre-aggregrate the number of reported
<code class="language-plaintext highlighter-rouge">tuples.received</code> values across all bolt instances that run on a particular Storm node.</p>
<p>To perform this per-host aggregation on the fly we must add the following lines to <code class="language-plaintext highlighter-rouge">/etc/carbon/aggregation-rules.conf</code>.
With those settings whenever we are sending a metric such as
<code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.storm-node01.tuples.received.count</code> from any bolt instance running on <code class="language-plaintext highlighter-rouge">storm-node01</code> to
Graphite (more correctly, its <code class="language-plaintext highlighter-rouge">carbon-aggregator</code> daemon), it will aggregate (here: <code class="language-plaintext highlighter-rouge">sum</code>) all such update messages for
<code class="language-plaintext highlighter-rouge">storm-node01</code> into a single, aggregated update message every 10 seconds for that server.</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># File: /etc/carbon/aggregation-rules.conf
<env>.apps.<app>.<server>.all.tuples.received.count (10) = sum <env>.apps.<app>.<server>.tuples.received.count
<env>.apps.<app>.<server>.all.tuples.received.m1_rate (10) = sum <env>.apps.<app>.<server>.tuples.received.m1_rate
<env>.apps.<app>.<server>.all.tuples.received.m5_rate (10) = sum <env>.apps.<app>.<server>.tuples.received.m5_rate
<env>.apps.<app>.<server>.all.tuples.received.m15_rate (10) = sum <env>.apps.<app>.<server>.tuples.received.m15_rate
<env>.apps.<app>.<server>.all.tuples.received.mean_rate (10) = sum <env>.apps.<app>.<server>.tuples.received.mean_rate
</code></pre></div></div>
<p>Lastly, make sure that the <code class="language-plaintext highlighter-rouge">carbon-aggregator</code> daemon is actually enabled in your <code class="language-plaintext highlighter-rouge">/etc/carbon/carbon.conf</code> and
configured to receive incoming data on its <code class="language-plaintext highlighter-rouge">LINE_RECEIVER_PORT</code> at <code class="language-plaintext highlighter-rouge">2023/tcp</code>. Also, make sure it sends its aggregates
to the <code class="language-plaintext highlighter-rouge">PICKLE_RECEIVER_PORT</code> of <code class="language-plaintext highlighter-rouge">carbon-cache</code> (port <code class="language-plaintext highlighter-rouge">2004/tcp</code>). See the <code class="language-plaintext highlighter-rouge">[aggregator]</code> section.</p>
<p>Example configuration snippet:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># File: /etc/carbon/carbon.conf
# ...snipp...
[aggregator]
LINE_RECEIVER_INTERFACE = 0.0.0.0
LINE_RECEIVER_PORT = 2023
PICKLE_RECEIVER_INTERFACE = 0.0.0.0
PICKLE_RECEIVER_PORT = 2024
DESTINATIONS = 127.0.0.1:2004 # <<< this points to the carbon-cache pickle port
# ...snipp...
</code></pre></div></div>
<p>Don’t forget to restart <code class="language-plaintext highlighter-rouge">carbon-aggregator</code> after changing its configuration:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ sudo service carbon-aggregator restart
</code></pre></div></div>
<h3 id="other-important-graphite-settings">Other important Graphite settings</h3>
<p>You may also want to check the values of the following Carbon settings in <code class="language-plaintext highlighter-rouge">/etc/carbon/carbon.conf</code>, particularly if you
are sending a lot of different metrics (= high number of metrics such as <code class="language-plaintext highlighter-rouge">my.foo</code> and <code class="language-plaintext highlighter-rouge">my.bar</code>) and/or a lot of metric
update messages per second (= high number of incoming metric updates for <code class="language-plaintext highlighter-rouge">my.foo</code>).</p>
<p>Whether or not you need to tune those settings depends on your specific use case. As a rule of thumb: The more Storm
nodes you have, the higher the topology’s parallelism and the higher your data volume, the more likely you will need to
optimize those settings. If you are not sure, leave them at their defaults and revisit later.</p>
<div class="note">
Note: I'd say the most important parameters at the very beginning are <tt>MAX_CREATES_PER_MINUTE</tt> (you might hit
this particularly when your topology starts to submit metrics for the very first time) and
<tt>MAX_UPDATES_PER_SECOND</tt>.
</div>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># File: /etc/carbon/carbon.conf
[cache]
# Limit the size of the cache to avoid swapping or becoming CPU bound.
# Sorts and serving cache queries gets more expensive as the cache grows.
# Use the value "inf" (infinity) for an unlimited cache size.
MAX_CACHE_SIZE = inf
# Limits the number of whisper update_many() calls per second, which effectively
# means the number of write requests sent to the disk. This is intended to
# prevent over-utilizing the disk and thus starving the rest of the system.
# When the rate of required updates exceeds this, then carbon's caching will
# take effect and increase the overall throughput accordingly.
MAX_UPDATES_PER_SECOND = 500
# Softly limits the number of whisper files that get created each minute.
# Setting this value low (like at 50) is a good way to ensure your graphite
# system will not be adversely impacted when a bunch of new metrics are
# sent to it. The trade off is that it will take much longer for those metrics'
# database files to all get created and thus longer until the data becomes usable.
# Setting this value high (like "inf" for infinity) will cause graphite to create
# the files quickly but at the risk of slowing I/O down considerably for a while.
MAX_CREATES_PER_MINUTE = 50
[aggregator]
# This is the maximum number of datapoints that can be queued up
# for a single destination. Once this limit is hit, we will
# stop accepting new data if USE_FLOW_CONTROL is True, otherwise
# we will drop any subsequently received datapoints.
MAX_QUEUE_SIZE = 10000
# Set this to False to drop datapoints when any send queue (sending datapoints
# to a downstream carbon daemon) hits MAX_QUEUE_SIZE. If this is True (the
# default) then sockets over which metrics are received will temporarily stop accepting
# data until the send queues fall below 80% MAX_QUEUE_SIZE.
USE_FLOW_CONTROL = True
# This defines the maximum "message size" between carbon daemons.
# You shouldn't need to tune this unless you really know what you're doing.
MAX_DATAPOINTS_PER_MESSAGE = 500
# This defines how many datapoints the aggregator remembers for
# each metric. Aggregation only happens for datapoints that fall in
# the past MAX_AGGREGATION_INTERVALS * intervalSize seconds.
MAX_AGGREGATION_INTERVALS = 5
</code></pre></div></div>
<h2 id="configuring-your-storm-code">Configuring your Storm code</h2>
<h3 id="add-the-metrics-library-to-your-storm-code-project">Add the Metrics library to your Storm code project</h3>
<p><em>The instructions below are for Gradle but it is straight-forward to adapt them to Maven if that’s your tool of choice.</em></p>
<p>Now that we have finished the Graphite setup we can turn our attention to augmenting our Storm code to work with
Graphite. Make sure <code class="language-plaintext highlighter-rouge">build.gradle</code> in your Storm code project looks similar to the following:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code># File: build.gradle
buildscript {
repositories {
mavenCentral()
}
dependencies {
// see https://github.com/musketyr/gradle-fatjar-plugin
classpath 'eu.appsatori:gradle-fatjar-plugin:0.2-rc1'
}
}
apply plugin: 'java'
apply plugin: 'fatjar'
// ...other plugins may follow here...
// We use JDK 6.
sourceCompatibility = 1.6
targetCompatibility = 1.6
group = 'com.miguno.storm.graphitedemo'
version = '0.1.0-SNAPSHOT'
repositories {
mavenCentral()
// required for Storm jars
mavenRepo url: "http://clojars.org/repo"
}
dependencies {
// Metrics library for reporting to Graphite
compile 'com.codahale.metrics:metrics-core:3.0.1'
compile 'com.codahale.metrics:metrics-annotation:3.0.1'
compile 'com.codahale.metrics:metrics-graphite:3.0.1'
// Storm
compile 'storm:storm:0.9.0-rc2', {
ext {
// Storm puts its own jar files on the CLASSPATH of a running topology by itself,
// and therefore does not want you to re-bundle Storm's class files with your
// topology jar.
fatJarExclude = true
}
}
// ...other dependencies may follow here...
}
// ...other gradle settings may follow here...
</code></pre></div></div>
<p>You can then run the usual gradle commands to compile, test and package your code. Particularly, you can now run:</p>
<div class="language-plaintext highlighter-rouge"><div class="highlight"><pre class="highlight"><code>$ gradle clean fatJar
</code></pre></div></div>
<p>This command will create a <em>fat jar</em> (also called <em>uber jar</em>) of your Storm topology code, which will be stored under
<code class="language-plaintext highlighter-rouge">build/libs/*.jar</code> by default. You can use this jar file to submit your topology to Storm via the <code class="language-plaintext highlighter-rouge">storm jar</code> command.
See the section on how to
<a href="/tutorials/running-multi-node-storm-cluster/#build-a-correct-standalone--fat-jar-file-of-my-storm-code">build a correct standalone jar file of your Storm code</a>
in my Storm multi-node cluster tutorial for details.</p>
<h3 id="sending-metrics-from-a-storm-bolt-to-graphite">Sending metrics from a Storm bolt to Graphite</h3>
<p>In this section we will augment a Storm bolt (spouts will work just the same) to report our <code class="language-plaintext highlighter-rouge">tuples.received</code> metrics to
Graphite.</p>
<p>Our bolt, i.e. its instances, will send this metric under the Graphite namespace
<code class="language-plaintext highlighter-rouge">production.apps.graphitedemo.HOSTNAME.tuples.received.*</code> every 10 seconds to the <code class="language-plaintext highlighter-rouge">carbon-aggregator</code> daemon running at
<code class="language-plaintext highlighter-rouge">your.graphite.server.com:2023/tcp</code>.</p>
<p>The <strong>key points</strong> of the code below are, firstly, the use of a <code class="language-plaintext highlighter-rouge">transient private</code> field for the <code class="language-plaintext highlighter-rouge">Meter</code> instance. If
you do not make the field <code class="language-plaintext highlighter-rouge">transient</code> Storm will throw a <code class="language-plaintext highlighter-rouge">NotSerializableException</code> during runtime. This is because
Storm will serialize the code that a Storm worker needs to execute and ship it to the worker via the network. For this
reason the code of our bolt will initialize the <code class="language-plaintext highlighter-rouge">Meter</code> instance during the <code class="language-plaintext highlighter-rouge">prepare()</code> phase of a bolt instance, which
ensures that the <code class="language-plaintext highlighter-rouge">Meter</code> instance is set up before the first tuples arrive at the bolt instance. So this part
achieves proper <em>counting</em> of the tuples.</p>
<div class="note">
Note: By the way, do not try to make the field a <tt>static</tt>. While this will prevent the
<tt>NotSerializableException</tt> it will also result in all instances of the bolt running on the same JVM will share
the same <tt>Meter</tt> instance (and typically you will have many instances on many JVMs on many Storm nodes), which
will cause loss of metrics data. In this case you would observe in Graphite that the <tt>tuples.received.*</tt>
metrics would significantly under-count the actual number of incoming tuples. Been there, done that. :-)
</div>
<p>Secondly, the <code class="language-plaintext highlighter-rouge">prepare()</code> method also creates a new, dedicated <code class="language-plaintext highlighter-rouge">GraphiteReporter</code> instance for each bolt instance. This
achieves proper <em>reporting</em> of metric updates to Graphite.</p>
<div class="language-java highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="c1">// BoltThatAlsoReportsToGraphite.java</span>
<span class="kn">package</span> <span class="nn">com.miguno.storm.graphitedemo</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">com.codahale.metrics.Meter</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">com.codahale.metrics.MetricFilter</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">com.codahale.metrics.MetricRegistry</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">com.codahale.metrics.graphite.Graphite</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">com.codahale.metrics.graphite.GraphiteReporter</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">org.apache.log4j.Logger</span><span class="o">;</span>
<span class="c1">// ...other imports such as backtype.storm.* omitted for clarity...</span>
<span class="kn">import</span> <span class="nn">java.net.InetAddress</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">java.net.UnknownHostException</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">java.util.regex.Pattern</span><span class="o">;</span>
<span class="kd">public</span> <span class="kd">class</span> <span class="nc">BoltThatAlsoReportsToGraphite</span> <span class="kd">extends</span> <span class="nc">BaseBasicBolt</span> <span class="o">{</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="kd">final</span> <span class="nc">Logger</span> <span class="no">LOG</span> <span class="o">=</span> <span class="nc">Logger</span><span class="o">.</span><span class="na">getLogger</span><span class="o">(</span><span class="nc">BoltThatAlsoReportsToGraphite</span><span class="o">.</span><span class="na">class</span><span class="o">);</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="kd">final</span> <span class="nc">String</span> <span class="no">GRAPHITE_HOST</span> <span class="o">=</span> <span class="s">"your.graphite.server.com"</span><span class="o">;</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="kd">final</span> <span class="kt">int</span> <span class="no">CARBON_AGGREGATOR_LINE_RECEIVER_PORT</span> <span class="o">=</span> <span class="mi">2023</span><span class="o">;</span>
<span class="c1">// The following value must match carbon-cache's storage-schemas.conf!</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="kd">final</span> <span class="kt">int</span> <span class="no">GRAPHITE_REPORT_INTERVAL_IN_SECONDS</span> <span class="o">=</span> <span class="mi">10</span><span class="o">;</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="kd">final</span> <span class="nc">String</span> <span class="no">GRAPHITE_METRICS_NAMESPACE_PREFIX</span> <span class="o">=</span>
<span class="s">"production.apps.graphitedemo"</span><span class="o">;</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="kd">final</span> <span class="nc">Pattern</span> <span class="n">hostnamePattern</span> <span class="o">=</span>
<span class="nc">Pattern</span><span class="o">.</span><span class="na">compile</span><span class="o">(</span><span class="s">"^[a-zA-Z0-9][a-zA-Z0-9-]*(\\.([a-zA-Z0-9][a-zA-Z0-9-]*))*$"</span><span class="o">);</span>
<span class="kd">private</span> <span class="kd">transient</span> <span class="nc">Meter</span> <span class="n">tuplesReceived</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="nc">Map</span> <span class="n">stormConf</span><span class="o">,</span> <span class="nc">TopologyContext</span> <span class="n">context</span><span class="o">)</span> <span class="o">{</span>
<span class="n">initializeMetricReporting</span><span class="o">();</span>
<span class="o">}</span>
<span class="kd">private</span> <span class="kt">void</span> <span class="nf">initializeMetricReporting</span><span class="o">()</span> <span class="o">{</span>
<span class="kd">final</span> <span class="nc">MetricRegistry</span> <span class="n">registry</span> <span class="o">=</span> <span class="k">new</span> <span class="nc">MetricRegistry</span><span class="o">();</span>
<span class="kd">final</span> <span class="nc">Graphite</span> <span class="n">graphite</span> <span class="o">=</span> <span class="k">new</span> <span class="nc">Graphite</span><span class="o">(</span><span class="k">new</span> <span class="nc">InetSocketAddress</span><span class="o">(</span><span class="no">GRAPHITE_HOST</span><span class="o">,</span>
<span class="no">CARBON_AGGREGATOR_LINE_RECEIVER_PORT</span><span class="o">));</span>
<span class="kd">final</span> <span class="nc">GraphiteReporter</span> <span class="n">reporter</span> <span class="o">=</span> <span class="nc">GraphiteReporter</span><span class="o">.</span><span class="na">forRegistry</span><span class="o">(</span><span class="n">registry</span><span class="o">)</span>
<span class="o">.</span><span class="na">prefixedWith</span><span class="o">(</span><span class="n">metricsPath</span><span class="o">())</span>
<span class="o">.</span><span class="na">convertRatesTo</span><span class="o">(</span><span class="nc">TimeUnit</span><span class="o">.</span><span class="na">SECONDS</span><span class="o">)</span>
<span class="o">.</span><span class="na">convertDurationsTo</span><span class="o">(</span><span class="nc">TimeUnit</span><span class="o">.</span><span class="na">MILLISECONDS</span><span class="o">)</span>
<span class="o">.</span><span class="na">filter</span><span class="o">(</span><span class="nc">MetricFilter</span><span class="o">.</span><span class="na">ALL</span><span class="o">)</span>
<span class="o">.</span><span class="na">build</span><span class="o">(</span><span class="n">graphite</span><span class="o">);</span>
<span class="n">reporter</span><span class="o">.</span><span class="na">start</span><span class="o">(</span><span class="no">GRAPHITE_REPORT_INTERVAL_IN_SECONDS</span><span class="o">,</span> <span class="nc">TimeUnit</span><span class="o">.</span><span class="na">SECONDS</span><span class="o">);</span>
<span class="n">tuplesReceived</span> <span class="o">=</span> <span class="n">registry</span><span class="o">.</span><span class="na">meter</span><span class="o">(</span><span class="nc">MetricRegistry</span><span class="o">.</span><span class="na">name</span><span class="o">(</span><span class="s">"tuples"</span><span class="o">,</span> <span class="s">"received"</span><span class="o">));</span>
<span class="o">}</span>
<span class="kd">private</span> <span class="nc">String</span> <span class="nf">metricsPath</span><span class="o">()</span> <span class="o">{</span>
<span class="kd">final</span> <span class="nc">String</span> <span class="n">myHostname</span> <span class="o">=</span> <span class="n">extractHostnameFromFQHN</span><span class="o">(</span><span class="n">detectHostname</span><span class="o">());</span>
<span class="k">return</span> <span class="no">GRAPHITE_METRICS_NAMESPACE_PREFIX</span> <span class="o">+</span> <span class="s">"."</span> <span class="o">+</span> <span class="n">myHostname</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="nc">Tuple</span> <span class="n">tuple</span><span class="o">,</span> <span class="nc">BasicOutputCollector</span> <span class="n">collector</span><span class="o">)</span> <span class="o">{</span>
<span class="n">tuplesReceived</span><span class="o">.</span><span class="na">mark</span><span class="o">();</span>
<span class="c1">// FYI: We do not need to explicitly ack() the tuple because we are extending</span>
<span class="c1">// BaseBasicBolt, which will automatically take care of that.</span>
<span class="o">}</span>
<span class="c1">// ...other bolt code may follow here...</span>
<span class="c1">//</span>
<span class="c1">// Helper methods to detect the hostname of the machine that</span>
<span class="c1">// executes this instance of a bolt. Normally you'd want to</span>
<span class="c1">// move this functionality into a separate class to adhere</span>
<span class="c1">// to the single responsibility principle.</span>
<span class="c1">//</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="nc">String</span> <span class="nf">detectHostname</span><span class="o">()</span> <span class="o">{</span>
<span class="nc">String</span> <span class="n">hostname</span> <span class="o">=</span> <span class="s">"hostname-could-not-be-detected"</span><span class="o">;</span>
<span class="k">try</span> <span class="o">{</span>
<span class="n">hostname</span> <span class="o">=</span> <span class="nc">InetAddress</span><span class="o">.</span><span class="na">getLocalHost</span><span class="o">().</span><span class="na">getHostName</span><span class="o">();</span>
<span class="o">}</span>
<span class="k">catch</span> <span class="o">(</span><span class="nc">UnknownHostException</span> <span class="n">e</span><span class="o">)</span> <span class="o">{</span>
<span class="no">LOG</span><span class="o">.</span><span class="na">error</span><span class="o">(</span><span class="s">"Could not determine hostname"</span><span class="o">);</span>
<span class="o">}</span>
<span class="k">return</span> <span class="n">hostname</span><span class="o">;</span>
<span class="o">}</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="nc">String</span> <span class="nf">extractHostnameFromFQHN</span><span class="o">(</span><span class="nc">String</span> <span class="n">fqhn</span><span class="o">)</span> <span class="o">{</span>
<span class="k">if</span> <span class="o">(</span><span class="n">hostnamePattern</span><span class="o">.</span><span class="na">matcher</span><span class="o">(</span><span class="n">fqhn</span><span class="o">).</span><span class="na">matches</span><span class="o">())</span> <span class="o">{</span>
<span class="k">if</span> <span class="o">(</span><span class="n">fqhn</span><span class="o">.</span><span class="na">contains</span><span class="o">(</span><span class="s">"."</span><span class="o">))</span> <span class="o">{</span>
<span class="k">return</span> <span class="n">fqhn</span><span class="o">.</span><span class="na">split</span><span class="o">(</span><span class="s">"\\."</span><span class="o">)[</span><span class="mi">0</span><span class="o">];</span>
<span class="o">}</span>
<span class="k">else</span> <span class="o">{</span>
<span class="k">return</span> <span class="n">fqhn</span><span class="o">;</span>
<span class="o">}</span>
<span class="o">}</span>
<span class="k">else</span> <span class="o">{</span>
<span class="c1">// We want to return the input as-is</span>
<span class="c1">// when it is not a valid hostname/FQHN.</span>
<span class="k">return</span> <span class="n">fqhn</span><span class="o">;</span>
<span class="o">}</span>
<span class="o">}</span>
<span class="o">}</span>
</code></pre></div></div>
<p>That’s it! Your Storm bolt instances will report their respective counts of received tuples to Graphite every 10
seconds.</p>
<h1 id="summary">Summary</h1>
<p>At this point you should have successfully married Storm with Graphite, and also learned a few basics about how
Graphite and Storm work along the way. Now you can begin creating graphs and dashboards for your Storm applications,
which was the reason to do all this in the first place, right?</p>
<p>Enjoy!</p>
<h1 id="appendix">Appendix</h1>
<h2 id="where-to-go-from-here">Where to go from here</h2>
<ul>
<li>Want to install and configure Graphite automatically? Take a look at my
<a href="https://github.com/miguno/puppet-graphite">puppet-graphite</a> module for <a href="http://puppetlabs.com/">Puppet</a>. See also
my previous post on
<a href="/blog/2013/06/06/installing-and-running-graphite-via-rpm-and-supervisord/">Installing and Running Graphite via RPM and Supervisord</a> for an alternative, manual installation approach.</li>
<li>Storm exposes a plethora of built-in metrics that greatly augment the application-level metrics we described in this
article. In 2015 we open sourced <a href="https://github.com/verisign/storm-graphite">storm-graphite</a>, which automatically
forwards these built-in metrics from Storm to Graphite. You can enable storm-graphite globally in your Storm cluster
or selectively for only a subset of your topologies.</li>
<li>You should start sending <em>system metrics</em> (CPU, memory and such) to Graphite, too. This allows you to correlate the
performance of your Storm topologies with the health of the machines in the cluster. Very helpful for detecting and
fixing bottlenecks! There are a couple of tools that can collect these system metrics for you and forward them to
Graphite. One of those tools is <a href="https://github.com/BrightcoveOS/Diamond">Diamond</a>. Take a look at my
<a href="https://github.com/miguno/puppet-diamond">puppet-diamond</a> Puppet module to automatically install and configure
Diamond on your Storm cluster nodes.</li>
<li>Want to install and configure Storm automatically? I am about to release an automated deployment tool called
Wirbelsturm very soon, which will allow you to deploy software such as Storm and Kafka. Wirbelsturm is essentially a
curated collection of <a href="http://puppetlabs.com/">Puppet</a> modules (that can be used standalone, too) plus a ready-to-use
<a href="http://www.vagrantup.com/">Vagrant</a> setup to deploy machines locally and to, say, Amazon AWS. <code class="language-plaintext highlighter-rouge">puppet-graphite</code> and
<code class="language-plaintext highlighter-rouge">puppet-diamond</code> above are part of the package, by the way. Please stay tuned! In the meantime my tutorial
<a href="/tutorials/running-multi-node-storm-cluster/">Running a Multi-Node Storm Cluster</a> should get you started.</li>
</ul>
<h2 id="caveat-storm-samples-metrics-for-the-storm-ui">Caveat: Storm samples metrics for the Storm UI</h2>
<p>If you do want to compare values 1:1 between the Storm UI and Graphite please be aware that by default Storm samples
incoming tuples for computing stats. By default it uses a sampling rate of 0.05 (5%), which is an option configurable
through <code class="language-plaintext highlighter-rouge">Config.TOPOLOGY_STATS_SAMPLE_RATE</code>.</p>
<blockquote><p>The way it works is that if you choose a sampling rate of 0.05, it will pick a random element of the next 20 events in which to increase the count by 20. So if you have 20 tasks for that bolt, your stats could be off by +-380.</p><footer><strong>Nathan Marz on storm-user</strong> <cite><a href="https://groups.google.com/d/msg/storm-user/q40AQHCV1L4/-XrOmBIAAngJ">groups.google.com/d/msg/…</a></cite></footer></blockquote>
<p>To force Storm to count everything exactly to achieve accurate numbers at the cost of a big performance hit to your
topology you can set the sampling rate to 100%:</p>
<div class="language-java highlighter-rouge"><div class="highlight"><pre class="highlight"><code><span class="n">conf</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="nc">Config</span><span class="o">.</span><span class="na">TOPOLOGY_STATS_SAMPLE_RATE</span><span class="o">,</span> <span class="mf">1.0</span><span class="o">);</span> <span class="c1">// default is 0.05</span>
</code></pre></div></div>So you got your first distributed Storm cluster installed and have your first topologies up and running. Great! Now you want to integrate your Storm applications with your monitoring systems and begin tracking application-level metrics from your topologies. In this article I show you how to integrate Storm with the popular Graphite monitoring system. This, combined with the Storm UI, will provide you with actionable information to tune the performance of your topologies and also help you to track key business as well as technical metrics.