<!DOCTYPE html>
<!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]-->
<!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]-->
<!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1">
<title>Spark Streaming Custom Receivers - Spark 1.4.0 Documentation</title>
<link rel="stylesheet" href="css/bootstrap.min.css">
<style>
body {
padding-top: 60px;
padding-bottom: 40px;
}
</style>
<meta name="viewport" content="width=device-width">
<link rel="stylesheet" href="css/bootstrap-responsive.min.css">
<link rel="stylesheet" href="css/main.css">
<script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
<link rel="stylesheet" href="css/pygments-default.css">
</head>
<body>
<!--[if lt IE 7]>
<p class="chromeframe">You are using an outdated browser. <a href="http://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p>
<![endif]-->
<!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html -->
<div class="navbar navbar-fixed-top" id="topbar">
<div class="navbar-inner">
<div class="container">
<div class="brand"><a href="index.html">
<img src="img/spark-logo-hd.png" style="height:50px;"/></a><span class="version">1.4.0</span>
</div>
<ul class="nav">
<!--TODO(andyk): Add class="active" attribute to li some how.-->
<li><a href="index.html">Overview</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">Programming Guides<b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="quick-start.html">Quick Start</a></li>
<li><a href="programming-guide.html">Spark Programming Guide</a></li>
<li class="divider"></li>
<li><a href="streaming-programming-guide.html">Spark Streaming</a></li>
<li><a href="sql-programming-guide.html">DataFrames and SQL</a></li>
<li><a href="mllib-guide.html">MLlib (Machine Learning)</a></li>
<li><a href="graphx-programming-guide.html">GraphX (Graph Processing)</a></li>
<li><a href="bagel-programming-guide.html">Bagel (Pregel on Spark)</a></li>
<li><a href="sparkr.html">SparkR (R on Spark)</a></li>
</ul>
</li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">API Docs<b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="api/scala/index.html#org.apache.spark.package">Scala</a></li>
<li><a href="api/java/index.html">Java</a></li>
<li><a href="api/python/index.html">Python</a></li>
<li><a href="api/R/index.html">R</a></li>
</ul>
</li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">Deploying<b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="cluster-overview.html">Overview</a></li>
<li><a href="submitting-applications.html">Submitting Applications</a></li>
<li class="divider"></li>
<li><a href="spark-standalone.html">Spark Standalone</a></li>
<li><a href="running-on-mesos.html">Mesos</a></li>
<li><a href="running-on-yarn.html">YARN</a></li>
<li class="divider"></li>
<li><a href="ec2-scripts.html">Amazon EC2</a></li>
</ul>
</li>
<li class="dropdown">
<a href="api.html" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="configuration.html">Configuration</a></li>
<li><a href="monitoring.html">Monitoring</a></li>
<li><a href="tuning.html">Tuning Guide</a></li>
<li><a href="job-scheduling.html">Job Scheduling</a></li>
<li><a href="security.html">Security</a></li>
<li><a href="hardware-provisioning.html">Hardware Provisioning</a></li>
<li><a href="hadoop-third-party-distributions.html">3<sup>rd</sup>-Party Hadoop Distros</a></li>
<li class="divider"></li>
<li><a href="building-spark.html">Building Spark</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark">Contributing to Spark</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/SPARK/Supplemental+Spark+Projects">Supplemental Projects</a></li>
</ul>
</li>
</ul>
<!--<p class="navbar-text pull-right"><span class="version-text">v1.4.0</span></p>-->
</div>
</div>
</div>
<div class="container" id="content">
<h1 class="title">Spark Streaming Custom Receivers</h1>
<p>Spark Streaming can receive streaming data from any arbitrary data source beyond
the one’s for which it has in-built support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.).
This requires the developer to implement a <em>receiver</em> that is customized for receiving data from
the concerned data source. This guide walks through the process of implementing a custom receiver
and using it in a Spark Streaming application. Note that custom receivers can be implemented
in Scala or Java.</p>
<h2 id="implementing-a-custom-receiver">Implementing a Custom Receiver</h2>
<p>This starts with implementing a <strong>Receiver</strong>
(<a href="api/scala/index.html#org.apache.spark.streaming.receiver.Receiver">Scala doc</a>,
<a href="api/java/org/apache/spark/streaming/receiver/Receiver.html">Java doc</a>).
A custom receiver must extend this abstract class by implementing two methods</p>
<ul>
<li><code>onStart()</code>: Things to do to start receiving data.</li>
<li><code>onStop()</code>: Things to do to stop receiving data.</li>
</ul>
<p>Both <code>onStart()</code> and <code>onStop()</code> must not block indefinitely. Typically, <code>onStart()</code> would start the threads
that responsible for receiving the data and <code>onStop()</code> would ensure that the receiving by those threads
are stopped. The receiving threads can also use <code>isStopped()</code>, a <code>Receiver</code> method, to check whether they
should stop receiving data.</p>
<p>Once the data is received, that data can be stored inside Spark
by calling <code>store(data)</code>, which is a method provided by the Receiver class.
There are number of flavours of <code>store()</code> which allow you store the received data
record-at-a-time or as whole collection of objects / serialized bytes. Note that the flavour of
<code>store()</code> used to implemented a receiver affects its reliability and fault-tolerance semantics.
This is discussed <a href="#receiver-reliability">later</a> in more detail.</p>
<p>Any exception in the receiving threads should be caught and handled properly to avoid silent
failures of the receiver. <code>restart(<exception>)</code> will restart the receiver by
asynchronously calling <code>onStop()</code> and then calling <code>onStart()</code> after a delay.
<code>stop(<exception>)</code> will call <code>onStop()</code> and terminate the receiver. Also, <code>reportError(<error>)</code>
reports a error message to the driver (visible in the logs and UI) without stopping / restarting
the receiver.</p>
<p>The following is a custom receiver that receives a stream of text over a socket. It treats
‘\n’ delimited lines in the text stream as records and stores them with Spark. If the receiving thread
has any error connecting or receiving, the receiver is restarted to make another attempt to connect.</p>
<div class="codetabs">
<div data-lang="scala">
<div class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">class</span> <span class="nc">CustomReceiver</span><span class="o">(</span><span class="n">host</span><span class="k">:</span> <span class="kt">String</span><span class="o">,</span> <span class="n">port</span><span class="k">:</span> <span class="kt">Int</span><span class="o">)</span>
<span class="k">extends</span> <span class="nc">Receiver</span><span class="o">[</span><span class="kt">String</span><span class="o">](</span><span class="nc">StorageLevel</span><span class="o">.</span><span class="nc">MEMORY_AND_DISK_2</span><span class="o">)</span> <span class="k">with</span> <span class="nc">Logging</span> <span class="o">{</span>
<span class="k">def</span> <span class="n">onStart</span><span class="o">()</span> <span class="o">{</span>
<span class="c1">// Start the thread that receives data over a connection</span>
<span class="k">new</span> <span class="nc">Thread</span><span class="o">(</span><span class="s">"Socket Receiver"</span><span class="o">)</span> <span class="o">{</span>
<span class="k">override</span> <span class="k">def</span> <span class="n">run</span><span class="o">()</span> <span class="o">{</span> <span class="n">receive</span><span class="o">()</span> <span class="o">}</span>
<span class="o">}.</span><span class="n">start</span><span class="o">()</span>
<span class="o">}</span>
<span class="k">def</span> <span class="n">onStop</span><span class="o">()</span> <span class="o">{</span>
<span class="c1">// There is nothing much to do as the thread calling receive()</span>
<span class="c1">// is designed to stop by itself isStopped() returns false</span>
<span class="o">}</span>
<span class="cm">/** Create a socket connection and receive data until receiver is stopped */</span>
<span class="k">private</span> <span class="k">def</span> <span class="n">receive</span><span class="o">()</span> <span class="o">{</span>
<span class="k">var</span> <span class="n">socket</span><span class="k">:</span> <span class="kt">Socket</span> <span class="o">=</span> <span class="kc">null</span>
<span class="k">var</span> <span class="n">userInput</span><span class="k">:</span> <span class="kt">String</span> <span class="o">=</span> <span class="kc">null</span>
<span class="k">try</span> <span class="o">{</span>
<span class="c1">// Connect to host:port</span>
<span class="n">socket</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">Socket</span><span class="o">(</span><span class="n">host</span><span class="o">,</span> <span class="n">port</span><span class="o">)</span>
<span class="c1">// Until stopped or connection broken continue reading</span>
<span class="k">val</span> <span class="n">reader</span> <span class="k">=</span> <span class="k">new</span> <span class="nc">BufferedReader</span><span class="o">(</span><span class="k">new</span> <span class="nc">InputStreamReader</span><span class="o">(</span><span class="n">socket</span><span class="o">.</span><span class="n">getInputStream</span><span class="o">(),</span> <span class="s">"UTF-8"</span><span class="o">))</span>
<span class="n">userInput</span> <span class="k">=</span> <span class="n">reader</span><span class="o">.</span><span class="n">readLine</span><span class="o">()</span>
<span class="k">while</span><span class="o">(!</span><span class="n">isStopped</span> <span class="o">&&</span> <span class="n">userInput</span> <span class="o">!=</span> <span class="kc">null</span><span class="o">)</span> <span class="o">{</span>
<span class="n">store</span><span class="o">(</span><span class="n">userInput</span><span class="o">)</span>
<span class="n">userInput</span> <span class="k">=</span> <span class="n">reader</span><span class="o">.</span><span class="n">readLine</span><span class="o">()</span>
<span class="o">}</span>
<span class="n">reader</span><span class="o">.</span><span class="n">close</span><span class="o">()</span>
<span class="n">socket</span><span class="o">.</span><span class="n">close</span><span class="o">()</span>
<span class="c1">// Restart in an attempt to connect again when server is active again</span>
<span class="n">restart</span><span class="o">(</span><span class="s">"Trying to connect again"</span><span class="o">)</span>
<span class="o">}</span> <span class="k">catch</span> <span class="o">{</span>
<span class="k">case</span> <span class="n">e</span><span class="k">:</span> <span class="kt">java.net.ConnectException</span> <span class="o">=></span>
<span class="c1">// restart if could not connect to server</span>
<span class="n">restart</span><span class="o">(</span><span class="s">"Error connecting to "</span> <span class="o">+</span> <span class="n">host</span> <span class="o">+</span> <span class="s">":"</span> <span class="o">+</span> <span class="n">port</span><span class="o">,</span> <span class="n">e</span><span class="o">)</span>
<span class="k">case</span> <span class="n">t</span><span class="k">:</span> <span class="kt">Throwable</span> <span class="o">=></span>
<span class="c1">// restart if there is any other error</span>
<span class="n">restart</span><span class="o">(</span><span class="s">"Error receiving data"</span><span class="o">,</span> <span class="n">t</span><span class="o">)</span>
<span class="o">}</span>
<span class="o">}</span>
<span class="o">}</span></code></pre></div>
</div>
<div data-lang="java">
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">class</span> <span class="nc">JavaCustomReceiver</span> <span class="kd">extends</span> <span class="n">Receiver</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="o">{</span>
<span class="n">String</span> <span class="n">host</span> <span class="o">=</span> <span class="kc">null</span><span class="o">;</span>
<span class="kt">int</span> <span class="n">port</span> <span class="o">=</span> <span class="o">-</span><span class="mi">1</span><span class="o">;</span>
<span class="kd">public</span> <span class="nf">JavaCustomReceiver</span><span class="o">(</span><span class="n">String</span> <span class="n">host_</span> <span class="o">,</span> <span class="kt">int</span> <span class="n">port_</span><span class="o">)</span> <span class="o">{</span>
<span class="kd">super</span><span class="o">(</span><span class="n">StorageLevel</span><span class="o">.</span><span class="na">MEMORY_AND_DISK_2</span><span class="o">());</span>
<span class="n">host</span> <span class="o">=</span> <span class="n">host_</span><span class="o">;</span>
<span class="n">port</span> <span class="o">=</span> <span class="n">port_</span><span class="o">;</span>
<span class="o">}</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">onStart</span><span class="o">()</span> <span class="o">{</span>
<span class="c1">// Start the thread that receives data over a connection</span>
<span class="k">new</span> <span class="nf">Thread</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">run</span><span class="o">()</span> <span class="o">{</span>
<span class="n">receive</span><span class="o">();</span>
<span class="o">}</span>
<span class="o">}.</span><span class="na">start</span><span class="o">();</span>
<span class="o">}</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">onStop</span><span class="o">()</span> <span class="o">{</span>
<span class="c1">// There is nothing much to do as the thread calling receive()</span>
<span class="c1">// is designed to stop by itself isStopped() returns false</span>
<span class="o">}</span>
<span class="cm">/** Create a socket connection and receive data until receiver is stopped */</span>
<span class="kd">private</span> <span class="kt">void</span> <span class="nf">receive</span><span class="o">()</span> <span class="o">{</span>
<span class="n">Socket</span> <span class="n">socket</span> <span class="o">=</span> <span class="kc">null</span><span class="o">;</span>
<span class="n">String</span> <span class="n">userInput</span> <span class="o">=</span> <span class="kc">null</span><span class="o">;</span>
<span class="k">try</span> <span class="o">{</span>
<span class="c1">// connect to the server</span>
<span class="n">socket</span> <span class="o">=</span> <span class="k">new</span> <span class="nf">Socket</span><span class="o">(</span><span class="n">host</span><span class="o">,</span> <span class="n">port</span><span class="o">);</span>
<span class="n">BufferedReader</span> <span class="n">reader</span> <span class="o">=</span> <span class="k">new</span> <span class="nf">BufferedReader</span><span class="o">(</span><span class="k">new</span> <span class="nf">InputStreamReader</span><span class="o">(</span><span class="n">socket</span><span class="o">.</span><span class="na">getInputStream</span><span class="o">()));</span>
<span class="c1">// Until stopped or connection broken continue reading</span>
<span class="k">while</span> <span class="o">(!</span><span class="n">isStopped</span><span class="o">()</span> <span class="o">&&</span> <span class="o">(</span><span class="n">userInput</span> <span class="o">=</span> <span class="n">reader</span><span class="o">.</span><span class="na">readLine</span><span class="o">())</span> <span class="o">!=</span> <span class="kc">null</span><span class="o">)</span> <span class="o">{</span>
<span class="n">System</span><span class="o">.</span><span class="na">out</span><span class="o">.</span><span class="na">println</span><span class="o">(</span><span class="s">"Received data '"</span> <span class="o">+</span> <span class="n">userInput</span> <span class="o">+</span> <span class="s">"'"</span><span class="o">);</span>
<span class="n">store</span><span class="o">(</span><span class="n">userInput</span><span class="o">);</span>
<span class="o">}</span>
<span class="n">reader</span><span class="o">.</span><span class="na">close</span><span class="o">();</span>
<span class="n">socket</span><span class="o">.</span><span class="na">close</span><span class="o">();</span>
<span class="c1">// Restart in an attempt to connect again when server is active again</span>
<span class="n">restart</span><span class="o">(</span><span class="s">"Trying to connect again"</span><span class="o">);</span>
<span class="o">}</span> <span class="k">catch</span><span class="o">(</span><span class="n">ConnectException</span> <span class="n">ce</span><span class="o">)</span> <span class="o">{</span>
<span class="c1">// restart if could not connect to server</span>
<span class="n">restart</span><span class="o">(</span><span class="s">"Could not connect"</span><span class="o">,</span> <span class="n">ce</span><span class="o">);</span>
<span class="o">}</span> <span class="k">catch</span><span class="o">(</span><span class="n">Throwable</span> <span class="n">t</span><span class="o">)</span> <span class="o">{</span>
<span class="c1">// restart if there is any other error</span>
<span class="n">restart</span><span class="o">(</span><span class="s">"Error receiving data"</span><span class="o">,</span> <span class="n">t</span><span class="o">);</span>
<span class="o">}</span>
<span class="o">}</span>
<span class="o">}</span></code></pre></div>
</div>
</div>
<h2 id="using-the-custom-receiver-in-a-spark-streaming-application">Using the custom receiver in a Spark Streaming application</h2>
<p>The custom receiver can be used in a Spark Streaming application by using
<code>streamingContext.receiverStream(<instance of custom receiver>)</code>. This will create
input DStream using data received by the instance of custom receiver, as shown below</p>
<div class="codetabs">
<div data-lang="scala">
<div class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Assuming ssc is the StreamingContext</span>
<span class="k">val</span> <span class="n">customReceiverStream</span> <span class="k">=</span> <span class="n">ssc</span><span class="o">.</span><span class="n">receiverStream</span><span class="o">(</span><span class="k">new</span> <span class="nc">CustomReceiver</span><span class="o">(</span><span class="n">host</span><span class="o">,</span> <span class="n">port</span><span class="o">))</span>
<span class="k">val</span> <span class="n">words</span> <span class="k">=</span> <span class="n">lines</span><span class="o">.</span><span class="n">flatMap</span><span class="o">(</span><span class="k">_</span><span class="o">.</span><span class="n">split</span><span class="o">(</span><span class="s">" "</span><span class="o">))</span>
<span class="o">...</span></code></pre></div>
<p>The full source code is in the example <a href="https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala">CustomReceiver.scala</a>.</p>
</div>
<div data-lang="java">
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="c1">// Assuming ssc is the JavaStreamingContext</span>
<span class="n">JavaDStream</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">customReceiverStream</span> <span class="o">=</span> <span class="n">ssc</span><span class="o">.</span><span class="na">receiverStream</span><span class="o">(</span><span class="k">new</span> <span class="nf">JavaCustomReceiver</span><span class="o">(</span><span class="n">host</span><span class="o">,</span> <span class="n">port</span><span class="o">));</span>
<span class="n">JavaDStream</span><span class="o"><</span><span class="n">String</span><span class="o">></span> <span class="n">words</span> <span class="o">=</span> <span class="n">lines</span><span class="o">.</span><span class="na">flatMap</span><span class="o">(</span><span class="k">new</span> <span class="n">FlatMapFunction</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">String</span><span class="o">>()</span> <span class="o">{</span> <span class="o">...</span> <span class="o">});</span>
<span class="o">...</span></code></pre></div>
<p>The full source code is in the example <a href="https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java">JavaCustomReceiver.java</a>.</p>
</div>
</div>
<h2 id="receiver-reliability">Receiver Reliability</h2>
<p>As discussed in brief in the
<a href="streaming-programming-guide.html#receiver-reliability">Spark Streaming Programming Guide</a>,
there are two kinds of receivers based on their reliability and fault-tolerance semantics.</p>
<ol>
<li><em>Reliable Receiver</em> - For <em>reliable sources</em> that allow sent data to be acknowledged, a
<em>reliable receiver</em> correctly acknowledges to the source that the data has been received
and stored in Spark reliably (that is, replicated successfully). Usually,
implementing this receiver involves careful consideration of the semantics of source
acknowledgements.</li>
<li><em>Unreliable Receiver</em> - These are receivers for unreliable sources that do not support
acknowledging. Even for reliable sources, one may implement an unreliable receiver that
do not go into the complexity of acknowledging correctly.</li>
</ol>
<p>To implement a <em>reliable receiver</em>, you have to use <code>store(multiple-records)</code> to store data.
This flavour of <code>store</code> is a blocking call which returns only after all the given records have
been stored inside Spark. If the receiver’s configured storage level uses replication
(enabled by default), then this call returns after replication has completed.
Thus it ensures that the data is reliably stored, and the receiver can now acknowledge the
source appropriately. This ensures that no data is caused when the receiver fails in the middle
of replicating data – the buffered data will not be acknowledged and hence will be later resent
by the source.</p>
<p>An <em>unreliable receiver</em> does not have to implement any of this logic. It can simply receive
records from the source and insert them one-at-a-time using <code>store(single-record)</code>. While it does
not get the reliability guarantees of <code>store(multiple-records)</code>, it has the following advantages.</p>
<ul>
<li>The system takes care of chunking that data into appropriate sized blocks (look for block
interval in the <a href="streaming-programming-guide.html">Spark Streaming Programming Guide</a>).</li>
<li>The system takes care of controlling the receiving rates if the rate limits have been specified.</li>
<li>Because of these two, unreliable receivers are simpler to implement than reliable receivers.</li>
</ul>
<p>The following table summarizes the characteristics of both types of receivers</p>
<table class="table">
<tr>
<th>Receiver Type</th>
<th>Characteristics</th>
</tr>
<tr>
<td><b>Unreliable Receivers</b></td>
<td>
Simple to implement.<br />
System takes care of block generation and rate control.
No fault-tolerance guarantees, can lose data on receiver failure.
</td>
</tr>
<tr>
<td><b>Reliable Receivers</b></td>
<td>
Strong fault-tolerance guarantees, can ensure zero data loss.<br />
Block generation and rate control to be handled by the receiver implementation.<br />
Implementation complexity depends on the acknowledgement mechanisms of the source.
</td>
</tr>
<tr>
<td></td>
<td></td>
</tr>
</table>
<h2 id="implementing-and-using-a-custom-actor-based-receiver">Implementing and Using a Custom Actor-based Receiver</h2>
<p>Custom <a href="http://doc.akka.io/docs/akka/2.2.4/scala/actors.html">Akka Actors</a> can also be used to
receive data. The <a href="api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper"><code>ActorHelper</code></a>
trait can be applied on any Akka actor, which allows received data to be stored in Spark using
<code>store(...)</code> methods. The supervisor strategy of this actor can be configured to handle failures, etc.</p>
<div class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="k">class</span> <span class="nc">CustomActor</span> <span class="k">extends</span> <span class="nc">Actor</span> <span class="k">with</span> <span class="nc">ActorHelper</span> <span class="o">{</span>
<span class="k">def</span> <span class="n">receive</span> <span class="k">=</span> <span class="o">{</span>
<span class="k">case</span> <span class="n">data</span><span class="k">:</span> <span class="kt">String</span> <span class="o">=></span> <span class="n">store</span><span class="o">(</span><span class="n">data</span><span class="o">)</span>
<span class="o">}</span>
<span class="o">}</span></code></pre></div>
<p>And a new input stream can be created with this custom actor as</p>
<div class="highlight"><pre><code class="language-scala" data-lang="scala"><span class="c1">// Assuming ssc is the StreamingContext</span>
<span class="k">val</span> <span class="n">lines</span> <span class="k">=</span> <span class="n">ssc</span><span class="o">.</span><span class="n">actorStream</span><span class="o">[</span><span class="kt">String</span><span class="o">](</span><span class="nc">Props</span><span class="o">(</span><span class="k">new</span> <span class="nc">CustomActor</span><span class="o">()),</span> <span class="s">"CustomReceiver"</span><span class="o">)</span></code></pre></div>
<p>See <a href="https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala">ActorWordCount.scala</a>
for an end-to-end example.</p>
</div> <!-- /container -->
<script src="js/vendor/jquery-1.8.0.min.js"></script>
<script src="js/vendor/bootstrap.min.js"></script>
<script src="js/main.js"></script>
<!-- MathJax Section -->
<script type="text/x-mathjax-config">
MathJax.Hub.Config({
TeX: { equationNumbers: { autoNumber: "AMS" } }
});
</script>
<script>
// Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS.
// We could use "//cdn.mathjax...", but that won't support "file://".
(function(d, script) {
script = d.createElement('script');
script.type = 'text/javascript';
script.async = true;
script.onload = function(){
MathJax.Hub.Config({
tex2jax: {
inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ],
displayMath: [ ["$$","$$"], ["\\[", "\\]"] ],
processEscapes: true,
skipTags: ['script', 'noscript', 'style', 'textarea', 'pre']
}
});
};
script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') +
'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML';
d.getElementsByTagName('head')[0].appendChild(script);
}(document));
</script>
</body>
</html>