You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
- 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.
Copy file name to clipboardExpand all lines: docs/index.md
+3-3Lines changed: 3 additions & 3 deletions
Display the source diff
Display the rich diff
Original file line number
Diff line number
Diff line change
@@ -112,10 +112,10 @@ Note that on Windows, you need to set the environment variables on separate line
112
112
*[Shark](http://shark.cs.berkeley.edu): Apache Hive over Spark
113
113
*[Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here
114
114
*[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
117
117
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
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
This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver).
92
+
{% endhighlight %}
13
93
14
-
The following is a simple socket text-stream receiver.
94
+
</div>
95
+
<divdata-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
+
}
15
154
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
-
}
38
155
{% endhighlight %}
39
156
157
+
</div>
158
+
</div>
40
159
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.
42
160
161
+
### Using the custom receiver in a Spark Streaming application
43
162
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
45
166
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
+
<divclass="codetabs">
168
+
<divdata-lang="scala"markdown="1" >
49
169
50
170
{% highlight scala %}
51
-
class SocketTextStreamReceiver (host:String,
52
-
port:Int,
53
-
bytesToString: ByteString => String) extends Actor with Receiver {
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
+
...
62
175
{% endhighlight %}
63
176
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).
67
178
68
-
* First create a Spark streaming context with master url and batchduration.
179
+
</div>
180
+
<divdata-lang="java"markdown="1">
69
181
70
-
{% highlight scala %}
71
-
val ssc = new StreamingContext(master, "WordCountCustomStreamSource",
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() { ... });
186
+
...
73
187
{% endhighlight %}
74
188
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).
76
190
77
-
{% highlight scala %}
78
-
val lines = ssc.networkStream[String](new SocketTextStreamReceiver(
79
-
"localhost", 8445))
80
-
{% endhighlight %}
191
+
</div>
192
+
</div>
81
193
82
-
* OR Plug-in the actor as receiver into the spark streaming context and create a DStream.
83
194
84
-
{% highlight scala %}
85
-
val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
86
-
"localhost",8445, z => z.utf8String)),"SocketReceiver")
87
-
{% endhighlight %}
88
195
89
-
* Process it.
196
+
### Implementing and Using a Custom Actor-based Receiver
90
197
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.
94
202
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
+
}
97
209
{% endhighlight %}
98
210
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
109
212
110
213
{% 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")
119
216
{% endhighlight %}
120
217
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.
124
220
125
-
## References
126
221
127
-
1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html)
0 commit comments