Skip to content

Latest commit

 

History

History
221 lines (171 loc) · 7.66 KB

streaming-custom-receivers.md

File metadata and controls

221 lines (171 loc) · 7.66 KB
layout title
global
Spark Streaming Custom Receivers

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 receiver 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.

Implementing a Custom Receiver

This starts with implementing a Receiver. A custom receiver must extend this abstract class by implementing two methods

  • onStart(): Things to do to start receiving data.
  • onStop(): Things to do to stop receiving data.

Note that onStart() and onStop() must not block indefinitely. Typically, onStart() would start the threads that responsible for receiving the data and onStop() would ensure that the receiving by those threads are stopped. The receiving threads can also use isStopped(), a Receiver method, to check whether they should stop receiving data.

Once the data is received, that data can be stored inside Spark by calling store(data), which is a method provided by the Receiver class. There are number of flavours of store() which allow you store the received data record-at-a-time or as whole collection of objects / serialized bytes.

Any exception in the receiving threads should be caught and handled properly to avoid silent failures of the receiver. restart(<exception>) will restart the receiver by asynchronously calling onStop() and then calling onStart() after a delay. stop(<exception>) will call onStop() and terminate the receiver. Also, reportError(<error>) reports a error message to the driver (visible in the logs and UI) without stopping / restarting the receiver.

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.

{% highlight scala %}

class CustomReceiver(host: String, port: Int) extends ReceiverString with Logging {

def onStart() { // Start the thread that receives data over a connection new Thread("Socket Receiver") { override def run() { receive() } }.start() }

def onStop() { // There is nothing much to do as the thread calling receive() // is designed to stop by itself isStopped() returns false }

/** Create a socket connection and receive data until receiver is stopped */ private def receive() { var socket: Socket = null var userInput: String = null try { // Connect to host:port socket = new Socket(host, port)

 // Until stopped or connection broken continue reading
 val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8"))
 userInput = reader.readLine()
 while(!isStopped && userInput != null) {
   store(userInput)
   userInput = reader.readLine()
 }
 reader.close()
 socket.close()

 // Restart in an attempt to connect again when server is active again
 restart("Trying to connect again")
} catch {
 case e: java.net.ConnectException =>
   // restart if could not connect to server
   restart("Error connecting to " + host + ":" + port, e)
 case t: Throwable =>
   // restart if there is any other error
   restart("Error receiving data", t)
}

} }

{% endhighlight %}

{% highlight java %}

public class JavaCustomReceiver extends Receiver {

String host = null; int port = -1;

public JavaCustomReceiver(String host_ , int port_) { super(StorageLevel.MEMORY_AND_DISK_2()); host = host_; port = port_; }

public void onStart() { // Start the thread that receives data over a connection new Thread() { @Override public void run() { receive(); } }.start(); }

public void onStop() { // There is nothing much to do as the thread calling receive() // is designed to stop by itself isStopped() returns false }

/** Create a socket connection and receive data until receiver is stopped */ private void receive() { Socket socket = null; String userInput = null;

try {
  // connect to the server
  socket = new Socket(host, port);

  BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()));

  // Until stopped or connection broken continue reading
  while (!isStopped() && (userInput = reader.readLine()) != null) {
    System.out.println("Received data '" + userInput + "'");
    store(userInput);
  }
  reader.close();
  socket.close();

  // Restart in an attempt to connect again when server is active again
  restart("Trying to connect again");
} catch(ConnectException ce) {
  // restart if could not connect to server
  restart("Could not connect", ce);
} catch(Throwable t) {
  // restart if there is any other error
  restart("Error receiving data", t);
}

} }

{% endhighlight %}

Using the custom receiver in a Spark Streaming application

The custom receiver can be used in a Spark Streaming application by using streamingContext.receiverStream(<instance of custom receiver>). This will create input DStream using data received by the instance of custom receiver, as shown below

{% highlight scala %} // Assuming ssc is the StreamingContext val customReceiverStream = ssc.receiverStream(new CustomReceiver(host, port)) val words = lines.flatMap(_.split(" ")) ... {% endhighlight %}

The full source code is in the example CustomReceiver.scala.

{% highlight java %} // Assuming ssc is the JavaStreamingContext JavaDStream customReceiverStream = ssc.receiverStream(new JavaCustomReceiver(host, port)); JavaDStream words = lines.flatMap(new FlatMapFunction<String, String>() { ... }); ... {% endhighlight %}

The full source code is in the example JavaCustomReceiver.java.

Implementing and Using a Custom Actor-based Receiver

Custom Akka Actors can also be used to receive data. The ActorHelper trait can be applied on any Akka actor, which allows received data to be stored in Spark using store(...) methods. The supervisor strategy of this actor can be configured to handle failures, etc.

{% highlight scala %} class CustomActor extends Actor with ActorHelper { def receive = { case data: String => store(data) } } {% endhighlight %}

And a new input stream can be created with this custom actor as

{% highlight scala %} // Assuming ssc is the StreamingContext val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver") {% endhighlight %}

See ActorWordCount.scala for an end-to-end example.