org.apache.spark.streaming.receiver
Class Receiver<T>

Object
  extended by org.apache.spark.streaming.receiver.Receiver<T>
All Implemented Interfaces:
java.io.Serializable

public abstract class Receiver<T>
extends Object
implements scala.Serializable

:: DeveloperApi :: Abstract class of a receiver that can be run on worker nodes to receive external data. A custom receiver can be defined by defining the functions onStart() and onStop(). onStart() should define the setup steps necessary to start receiving data, and onStop() should define the cleanup steps necessary to stop receiving data. Exceptions while receiving can be handled either by restarting the receiver with restart(...) or stopped completely by stop(...) or

A custom receiver in Scala would look like this.


  class MyReceiver(storageLevel: StorageLevel) extends NetworkReceiver[String](storageLevel) {
      def onStart() {
          // Setup stuff (start threads, open sockets, etc.) to start receiving data.
          // Must start new thread to receive data, as onStart() must be non-blocking.

          // Call store(...) in those threads to store received data into Spark's memory.

          // Call stop(...), restart(...) or reportError(...) on any thread based on how
          // different errors needs to be handled.

          // See corresponding method documentation for more details
      }

      def onStop() {
          // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data.
      }
  }
 

A custom receiver in Java would look like this.


 class MyReceiver extends Receiver<String> {
     public MyReceiver(StorageLevel storageLevel) {
         super(storageLevel);
     }

     public void onStart() {
          // Setup stuff (start threads, open sockets, etc.) to start receiving data.
          // Must start new thread to receive data, as onStart() must be non-blocking.

          // Call store(...) in those threads to store received data into Spark's memory.

          // Call stop(...), restart(...) or reportError(...) on any thread based on how
          // different errors needs to be handled.

          // See corresponding method documentation for more details
     }

     public void onStop() {
          // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data.
     }
 }
 

See Also:
Serialized Form

Constructor Summary
Receiver(StorageLevel storageLevel)
           
 
Method Summary
 org.apache.spark.streaming.receiver.ReceiverSupervisor executor_()
          Handler object that runs the receiver.
 boolean isStarted()
          Check if the receiver has started or not.
 boolean isStopped()
          Check if receiver has been marked for stopping.
abstract  void onStart()
          This method is called by the system when the receiver is started.
abstract  void onStop()
          This method is called by the system when the receiver is stopped.
 scala.Option<String> preferredLocation()
          Override this to specify a preferred location (hostname).
 void reportError(String message, Throwable throwable)
          Report exceptions in receiving data.
 void restart(String message)
          Restart the receiver.
 void restart(String message, Throwable error)
          Restart the receiver.
 void restart(String message, Throwable error, int millisecond)
          Restart the receiver.
 void stop(String message)
          Stop the receiver completely.
 void stop(String message, Throwable error)
          Stop the receiver completely due to an exception
 StorageLevel storageLevel()
           
 void store(scala.collection.mutable.ArrayBuffer<T> dataBuffer)
          Store an ArrayBuffer of received data as a data block into Spark's memory.
 void store(scala.collection.mutable.ArrayBuffer<T> dataBuffer, Object metadata)
          Store an ArrayBuffer of received data as a data block into Spark's memory.
 void store(java.nio.ByteBuffer bytes)
          Store the bytes of received data as a data block into Spark's memory.
 void store(java.nio.ByteBuffer bytes, Object metadata)
          Store the bytes of received data as a data block into Spark's memory.
 void store(scala.collection.Iterator<T> dataIterator)
          Store an iterator of received data as a data block into Spark's memory.
 void store(java.util.Iterator<T> dataIterator)
          Store an iterator of received data as a data block into Spark's memory.
 void store(java.util.Iterator<T> dataIterator, Object metadata)
          Store an iterator of received data as a data block into Spark's memory.
 void store(scala.collection.Iterator<T> dataIterator, Object metadata)
          Store an iterator of received data as a data block into Spark's memory.
 void store(T dataItem)
          Store a single item of received data to Spark's memory.
 int streamId()
          Get the unique identifier the receiver input stream that this receiver is associated with.
 
Methods inherited from class Object
equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
 

Constructor Detail

Receiver

public Receiver(StorageLevel storageLevel)
Method Detail

storageLevel

public StorageLevel storageLevel()

onStart

public abstract void onStart()
This method is called by the system when the receiver is started. This function must initialize all resources (threads, buffers, etc.) necessary for receiving data. This function must be non-blocking, so receiving the data must occur on a different thread. Received data can be stored with Spark by calling store(data).

If there are errors in threads started here, then following options can be done (i) reportError(...) can be called to report the error to the driver. The receiving of data will continue uninterrupted. (ii) stop(...) can be called to stop receiving data. This will call onStop() to clear up all resources allocated (threads, buffers, etc.) during onStart(). (iii) restart(...) can be called to restart the receiver. This will call onStop() immediately, and then onStart() after a delay.


onStop

public abstract void onStop()
This method is called by the system when the receiver is stopped. All resources (threads, buffers, etc.) setup in onStart() must be cleaned up in this method.


preferredLocation

public scala.Option<String> preferredLocation()
Override this to specify a preferred location (hostname).


store

public void store(T dataItem)
Store a single item of received data to Spark's memory. These single items will be aggregated together into data blocks before being pushed into Spark's memory.

Parameters:
dataItem - (undocumented)

store

public void store(scala.collection.mutable.ArrayBuffer<T> dataBuffer)
Store an ArrayBuffer of received data as a data block into Spark's memory.


store

public void store(scala.collection.mutable.ArrayBuffer<T> dataBuffer,
                  Object metadata)
Store an ArrayBuffer of received data as a data block into Spark's memory. The metadata will be associated with this block of data for being used in the corresponding InputDStream.

Parameters:
dataBuffer - (undocumented)
metadata - (undocumented)

store

public void store(scala.collection.Iterator<T> dataIterator)
Store an iterator of received data as a data block into Spark's memory.


store

public void store(java.util.Iterator<T> dataIterator,
                  Object metadata)
Store an iterator of received data as a data block into Spark's memory. The metadata will be associated with this block of data for being used in the corresponding InputDStream.

Parameters:
dataIterator - (undocumented)
metadata - (undocumented)

store

public void store(java.util.Iterator<T> dataIterator)
Store an iterator of received data as a data block into Spark's memory.


store

public void store(scala.collection.Iterator<T> dataIterator,
                  Object metadata)
Store an iterator of received data as a data block into Spark's memory. The metadata will be associated with this block of data for being used in the corresponding InputDStream.

Parameters:
dataIterator - (undocumented)
metadata - (undocumented)

store

public void store(java.nio.ByteBuffer bytes)
Store the bytes of received data as a data block into Spark's memory. Note that the data in the ByteBuffer must be serialized using the same serializer that Spark is configured to use.

Parameters:
bytes - (undocumented)

store

public void store(java.nio.ByteBuffer bytes,
                  Object metadata)
Store the bytes of received data as a data block into Spark's memory. The metadata will be associated with this block of data for being used in the corresponding InputDStream.

Parameters:
bytes - (undocumented)
metadata - (undocumented)

reportError

public void reportError(String message,
                        Throwable throwable)
Report exceptions in receiving data.


restart

public void restart(String message)
Restart the receiver. This method schedules the restart and returns immediately. The stopping and subsequent starting of the receiver (by calling onStop() and onStart()) is performed asynchronously in a background thread. The delay between the stopping and the starting is defined by the Spark configuration spark.streaming.receiverRestartDelay. The message will be reported to the driver.

Parameters:
message - (undocumented)

restart

public void restart(String message,
                    Throwable error)
Restart the receiver. This method schedules the restart and returns immediately. The stopping and subsequent starting of the receiver (by calling onStop() and onStart()) is performed asynchronously in a background thread. The delay between the stopping and the starting is defined by the Spark configuration spark.streaming.receiverRestartDelay. The message and exception will be reported to the driver.

Parameters:
message - (undocumented)
error - (undocumented)

restart

public void restart(String message,
                    Throwable error,
                    int millisecond)
Restart the receiver. This method schedules the restart and returns immediately. The stopping and subsequent starting of the receiver (by calling onStop() and onStart()) is performed asynchronously in a background thread.

Parameters:
message - (undocumented)
error - (undocumented)
millisecond - (undocumented)

stop

public void stop(String message)
Stop the receiver completely.


stop

public void stop(String message,
                 Throwable error)
Stop the receiver completely due to an exception


isStarted

public boolean isStarted()
Check if the receiver has started or not.


isStopped

public boolean isStopped()
Check if receiver has been marked for stopping. Use this to identify when the receiving of data should be stopped.

Returns:
(undocumented)

streamId

public int streamId()
Get the unique identifier the receiver input stream that this receiver is associated with.

Returns:
(undocumented)

executor_

public org.apache.spark.streaming.receiver.ReceiverSupervisor executor_()
Handler object that runs the receiver. This is instantiated lazily in the worker.