Skip to content

Commit a975a19

Browse files
committed
[SPARK-1504], [SPARK-1505], [SPARK-1558] Updated Spark Streaming guide
- SPARK-1558: Updated custom receiver guide to match it with the new API - SPARK-1504: Added deployment and monitoring subsection to streaming - SPARK-1505: Added migration guide for migrating from 0.9.x and below to Spark 1.0 - Updated various Java streaming examples to use JavaReceiverInputDStream to highlight the API change. - Removed the requirement for cleaner ttl from streaming guide Author: Tathagata Das <[email protected]> Closes #652 from tdas/doc-fix and squashes the following commits: cb4f4b7 [Tathagata Das] Possible fix for flaky graceful shutdown test. ab71f7f [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into doc-fix 8d6ff9b [Tathagata Das] Addded migration guide to Spark Streaming. 7d171df [Tathagata Das] Added reference to JavaReceiverInputStream in examples and streaming guide. 49edd7c [Tathagata Das] Change java doc links to use Java docs. 11528d7 [Tathagata Das] Updated links on index page. ff80970 [Tathagata Das] More updates to streaming guide. 4dc42e9 [Tathagata Das] Added monitoring and other documentation in the streaming guide. 14c6564 [Tathagata Das] Updated custom receiver guide.
1 parent 3292e2a commit a975a19

File tree

10 files changed

+360
-146
lines changed

10 files changed

+360
-146
lines changed

docs/configuration.md

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -462,7 +462,7 @@ Apart from these, the following properties are also available, and may be useful
462462
<td>(infinite)</td>
463463
<td>
464464
Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.).
465-
Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is
465+
Periodic cleanups will ensure that metadata older than this duration will be forgotten. This is
466466
useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming
467467
applications). Note that any RDD that persists in memory for more than this duration will be cleared as well.
468468
</td>
@@ -471,8 +471,8 @@ Apart from these, the following properties are also available, and may be useful
471471
<td>spark.streaming.blockInterval</td>
472472
<td>200</td>
473473
<td>
474-
Duration (milliseconds) of how long to batch new objects coming from network receivers used
475-
in Spark Streaming.
474+
Interval (milliseconds) at which data received by Spark Streaming receivers is coalesced
475+
into blocks of data before storing them in Spark.
476476
</td>
477477
</tr>
478478
<tr>

docs/index.md

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -112,10 +112,10 @@ Note that on Windows, you need to set the environment variables on separate line
112112
* [Shark](http://shark.cs.berkeley.edu): Apache Hive over Spark
113113
* [Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here
114114
* [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and
115-
exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012),
116-
[slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are
115+
exercises about Spark, Shark, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/),
116+
[slides](http://ampcamp.berkeley.edu/3/) and [exercises](http://ampcamp.berkeley.edu/3/exercises/) are
117117
available online for free.
118-
* [Code Examples](http://spark.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/spark/tree/master/examples/src/main/scala/) of Spark
118+
* [Code Examples](http://spark.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/spark/tree/master/examples/src/main/scala/org/apache/spark/) of Spark
119119
* [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf)
120120
* [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)
121121

docs/streaming-custom-receivers.md

Lines changed: 183 additions & 90 deletions
Original file line numberDiff line numberDiff line change
@@ -3,126 +3,219 @@ layout: global
33
title: Spark Streaming Custom Receivers
44
---
55

6-
A "Spark Streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need.
6+
Spark Streaming can receive streaming data from any arbitrary data source beyond
7+
the one's for which it has in-built support (that is, beyond Flume, Kafka, files, sockets, etc.).
8+
This requires the developer to implement a *receiver* that is customized for receiving data from
9+
the concerned data source. This guide walks through the process of implementing a custom receiver
10+
and using it in a Spark Streaming application.
11+
12+
### Implementing a Custom Receiver
13+
14+
This starts with implementing a [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver).
15+
A custom receiver must extend this abstract class by implementing two methods
16+
- `onStart()`: Things to do to start receiving data.
17+
- `onStop()`: Things to do to stop receiving data.
18+
19+
Note that `onStart()` and `onStop()` must not block indefinitely. Typically, onStart() would start the threads
20+
that responsible for receiving the data and `onStop()` would ensure that the receiving by those threads
21+
are stopped. The receiving threads can also use `isStopped()`, a `Receiver` method, to check whether they
22+
should stop receiving data.
23+
24+
Once the data is received, that data can be stored inside Spark
25+
by calling `store(data)`, which is a method provided by the
26+
[Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) class.
27+
There are number of flavours of `store()` which allow you store the received data
28+
record-at-a-time or as whole collection of objects / serialized bytes.
29+
30+
Any exception in the receiving threads should be caught and handled properly to avoid silent
31+
failures of the receiver. `restart(<exception>)` will restart the receiver by
32+
asynchronously calling `onStop()` and then calling `onStart()` after a delay.
33+
`stop(<exception>)` will call `onStop()` and terminate the receiver. Also, `reportError(<error>)`
34+
reports a error message to the driver (visible in the logs and UI) without stopping / restarting
35+
the receiver.
36+
37+
The following is a custom receiver that receives a stream of text over a socket. It treats
38+
'\n' delimited lines in the text stream as records and stores them with Spark. If the receiving thread
39+
has any error connecting or receiving, the receiver is restarted to make another attempt to connect.
40+
41+
<div class="codetabs">
42+
<div data-lang="scala" markdown="1" >
743

8-
This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application.
44+
{% highlight scala %}
945

10-
### Writing a Simple Receiver
46+
class CustomReceiver(host: String, port: Int)
47+
extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging {
48+
49+
def onStart() {
50+
// Start the thread that receives data over a connection
51+
new Thread("Socket Receiver") {
52+
override def run() { receive() }
53+
}.start()
54+
}
55+
56+
def onStop() {
57+
// There is nothing much to do as the thread calling receive()
58+
// is designed to stop by itself isStopped() returns false
59+
}
60+
61+
/** Create a socket connection and receive data until receiver is stopped */
62+
private def receive() {
63+
var socket: Socket = null
64+
var userInput: String = null
65+
try {
66+
// Connect to host:port
67+
socket = new Socket(host, port)
68+
69+
// Until stopped or connection broken continue reading
70+
val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8"))
71+
userInput = reader.readLine()
72+
while(!isStopped && userInput != null) {
73+
store(userInput)
74+
userInput = reader.readLine()
75+
}
76+
reader.close()
77+
socket.close()
78+
79+
// Restart in an attempt to connect again when server is active again
80+
restart("Trying to connect again")
81+
} catch {
82+
case e: java.net.ConnectException =>
83+
// restart if could not connect to server
84+
restart("Error connecting to " + host + ":" + port, e)
85+
case t: Throwable =>
86+
// restart if there is any other error
87+
restart("Error receiving data", t)
88+
}
89+
}
90+
}
1191

12-
This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver).
92+
{% endhighlight %}
1393

14-
The following is a simple socket text-stream receiver.
94+
</div>
95+
<div data-lang="java" markdown="1">
96+
97+
{% highlight java %}
98+
99+
public class JavaCustomReceiver extends Receiver<String> {
100+
101+
String host = null;
102+
int port = -1;
103+
104+
public JavaCustomReceiver(String host_ , int port_) {
105+
super(StorageLevel.MEMORY_AND_DISK_2());
106+
host = host_;
107+
port = port_;
108+
}
109+
110+
public void onStart() {
111+
// Start the thread that receives data over a connection
112+
new Thread() {
113+
@Override public void run() {
114+
receive();
115+
}
116+
}.start();
117+
}
118+
119+
public void onStop() {
120+
// There is nothing much to do as the thread calling receive()
121+
// is designed to stop by itself isStopped() returns false
122+
}
123+
124+
/** Create a socket connection and receive data until receiver is stopped */
125+
private void receive() {
126+
Socket socket = null;
127+
String userInput = null;
128+
129+
try {
130+
// connect to the server
131+
socket = new Socket(host, port);
132+
133+
BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()));
134+
135+
// Until stopped or connection broken continue reading
136+
while (!isStopped() && (userInput = reader.readLine()) != null) {
137+
System.out.println("Received data '" + userInput + "'");
138+
store(userInput);
139+
}
140+
reader.close();
141+
socket.close();
142+
143+
// Restart in an attempt to connect again when server is active again
144+
restart("Trying to connect again");
145+
} catch(ConnectException ce) {
146+
// restart if could not connect to server
147+
restart("Could not connect", ce);
148+
} catch(Throwable t) {
149+
// restart if there is any other error
150+
restart("Error receiving data", t);
151+
}
152+
}
153+
}
15154

16-
{% highlight scala %}
17-
class SocketTextStreamReceiver(host: String, port: Int)
18-
extends NetworkReceiver[String]
19-
{
20-
protected lazy val blocksGenerator: BlockGenerator =
21-
new BlockGenerator(StorageLevel.MEMORY_ONLY_SER_2)
22-
23-
protected def onStart() = {
24-
blocksGenerator.start()
25-
val socket = new Socket(host, port)
26-
val dataInputStream = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8"))
27-
var data: String = dataInputStream.readLine()
28-
while (data != null) {
29-
blocksGenerator += data
30-
data = dataInputStream.readLine()
31-
}
32-
}
33-
34-
protected def onStop() {
35-
blocksGenerator.stop()
36-
}
37-
}
38155
{% endhighlight %}
39156

157+
</div>
158+
</div>
40159

41-
All we did here is extended NetworkReceiver and called blockGenerator's API method (i.e. +=) to push our blocks of data. Please refer to scala-docs of NetworkReceiver for more details.
42160

161+
### Using the custom receiver in a Spark Streaming application
43162

44-
### An Actor as Receiver
163+
The custom receiver can be used in a Spark Streaming application by using
164+
`streamingContext.receiverStream(<instance of custom receiver>)`. This will create
165+
input DStream using data received by the instance of custom receiver, as shown below
45166

46-
This starts with implementing [Actor](#References)
47-
48-
Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api.
167+
<div class="codetabs">
168+
<div data-lang="scala" markdown="1" >
49169

50170
{% highlight scala %}
51-
class SocketTextStreamReceiver (host:String,
52-
port:Int,
53-
bytesToString: ByteString => String) extends Actor with Receiver {
54-
55-
override def preStart = IOManager(context.system).connect(host, port)
56-
57-
def receive = {
58-
case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes))
59-
}
60-
61-
}
171+
// Assuming ssc is the StreamingContext
172+
val customReceiverStream = ssc.receiverStream(new CustomReceiver(host, port))
173+
val words = lines.flatMap(_.split(" "))
174+
...
62175
{% endhighlight %}
63176

64-
All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details.
65-
66-
### A Sample Spark Application
177+
The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala).
67178

68-
* First create a Spark streaming context with master url and batchduration.
179+
</div>
180+
<div data-lang="java" markdown="1">
69181

70-
{% highlight scala %}
71-
val ssc = new StreamingContext(master, "WordCountCustomStreamSource",
72-
Seconds(batchDuration))
182+
{% highlight java %}
183+
// Assuming ssc is the JavaStreamingContext
184+
JavaDStream<String> customReceiverStream = ssc.receiverStream(new JavaCustomReceiver(host, port));
185+
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() { ... });
186+
...
73187
{% endhighlight %}
74188

75-
* Plug-in the custom receiver into the spark streaming context and create a DStream.
189+
The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java).
76190

77-
{% highlight scala %}
78-
val lines = ssc.networkStream[String](new SocketTextStreamReceiver(
79-
"localhost", 8445))
80-
{% endhighlight %}
191+
</div>
192+
</div>
81193

82-
* OR Plug-in the actor as receiver into the spark streaming context and create a DStream.
83194

84-
{% highlight scala %}
85-
val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
86-
"localhost",8445, z => z.utf8String)),"SocketReceiver")
87-
{% endhighlight %}
88195

89-
* Process it.
196+
### Implementing and Using a Custom Actor-based Receiver
90197

91-
{% highlight scala %}
92-
val words = lines.flatMap(_.split(" "))
93-
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
198+
Custom [Akka Actors](http://doc.akka.io/docs/akka/2.2.4/scala/actors.html) can also be used to
199+
receive data. The [`ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper)
200+
trait can be applied on any Akka actor, which allows received data to be stored in Spark using
201+
`store(...)` methods. The supervisor strategy of this actor can be configured to handle failures, etc.
94202

95-
wordCounts.print()
96-
ssc.start()
203+
{% highlight scala %}
204+
class CustomActor extends Actor with ActorHelper {
205+
def receive = {
206+
case data: String => store(data)
207+
}
208+
}
97209
{% endhighlight %}
98210

99-
* After processing it, stream can be tested using the netcat utility.
100-
101-
$ nc -l localhost 8445
102-
hello world
103-
hello hello
104-
105-
106-
## Multiple Homogeneous/Heterogeneous Receivers.
107-
108-
A DStream union operation is provided for taking union on multiple input streams.
211+
And a new input stream can be created with this custom actor as
109212

110213
{% highlight scala %}
111-
val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
112-
"localhost",8445, z => z.utf8String)),"SocketReceiver")
113-
114-
// Another socket stream receiver
115-
val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
116-
"localhost",8446, z => z.utf8String)),"SocketReceiver")
117-
118-
val union = lines.union(lines2)
214+
// Assuming ssc is the StreamingContext
215+
val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver")
119216
{% endhighlight %}
120217

121-
Above stream can be easily process as described earlier.
122-
123-
_A more comprehensive example is provided in the spark streaming examples_
218+
See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala)
219+
for an end-to-end example.
124220

125-
## References
126221

127-
1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html)
128-
2.[NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver)

0 commit comments

Comments
 (0)