@@ -20,6 +20,7 @@ package org.apache.spark.streaming.receiver
2020import java .nio .ByteBuffer
2121
2222import scala .collection .mutable .ArrayBuffer
23+ import scala .collection .JavaConversions ._
2324
2425import org .apache .spark .storage .StorageLevel
2526
@@ -30,23 +31,25 @@ import org.apache.spark.storage.StorageLevel
3031 * and onStop() should define the cleanup steps necessary to stop receiving data. A custom
3132 * receiver would look something like this.
3233 *
33- * class MyReceiver(storageLevel) extends NetworkReceiver[String](storageLevel) {
34- * def onStart() {
35- * // Setup stuff (start threads, open sockets, etc.) to start receiving data.
36- * // Must start new thread to receive data, as onStart() must be non-blocking.
34+ * @example {{{
35+ * class MyReceiver(storageLevel: StorageLevel) extends NetworkReceiver[String](storageLevel) {
36+ * def onStart() {
37+ * // Setup stuff (start threads, open sockets, etc.) to start receiving data.
38+ * // Must start new thread to receive data, as onStart() must be non-blocking.
3739 *
38- * // Call store(...) in those threads to store received data into Spark's memory.
40+ * // Call store(...) in those threads to store received data into Spark's memory.
3941 *
40- * // Call stop(...), restart() or reportError(...) on any thread based on how
41- * // different errors should be handled.
42+ * // Call stop(...), restart() or reportError(...) on any thread based on how
43+ * // different errors should be handled.
4244 *
43- * // See corresponding method documentation for more details.
44- * }
45+ * // See corresponding method documentation for more details
46+ * }
4547 *
46- * def onStop() {
47- * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data.
48- * }
49- * }
48+ * def onStop() {
49+ * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data.
50+ * }
51+ * }
52+ * }}}
5053 */
5154abstract class NetworkReceiver [T ](val storageLevel : StorageLevel ) extends Serializable {
5255
@@ -80,43 +83,58 @@ abstract class NetworkReceiver[T](val storageLevel: StorageLevel) extends Serial
8083 executor.pushSingle(dataItem)
8184 }
8285
83- /** Store a sequence of received data into Spark's memory. */
86+ /** Store an ArrayBuffer of received data as a data block into Spark's memory. */
8487 def store (dataBuffer : ArrayBuffer [T ]) {
8588 executor.pushArrayBuffer(dataBuffer, None , None )
8689 }
8790
8891 /**
89- * Store a sequence of received data into Spark's memory.
92+ * Store an ArrayBuffer of received data as a data block into Spark's memory.
9093 * The metadata will be associated with this block of data
9194 * for being used in the corresponding InputDStream.
9295 */
9396 def store (dataBuffer : ArrayBuffer [T ], metadata : Any ) {
9497 executor.pushArrayBuffer(dataBuffer, Some (metadata), None )
9598 }
96- /** Store a sequence of received data into Spark's memory. */
99+
100+ /** Store a iterator of received data as a data block into Spark's memory. */
97101 def store (dataIterator : Iterator [T ]) {
98102 executor.pushIterator(dataIterator, None , None )
99103 }
100104
101105 /**
102- * Store a sequence of received data into Spark's memory.
106+ * Store a iterator of received data as a data block into Spark's memory.
107+ * The metadata will be associated with this block of data
108+ * for being used in the corresponding InputDStream.
109+ */
110+ def store (dataIterator : java.util.Iterator [T ], metadata : Any ) {
111+ executor.pushIterator(dataIterator, Some (metadata), None )
112+ }
113+
114+ /** Store a iterator of received data as a data block into Spark's memory. */
115+ def store (dataIterator : java.util.Iterator [T ]) {
116+ executor.pushIterator(dataIterator, None , None )
117+ }
118+
119+ /**
120+ * Store a iterator of received data as a data block into Spark's memory.
103121 * The metadata will be associated with this block of data
104122 * for being used in the corresponding InputDStream.
105123 */
106124 def store (dataIterator : Iterator [T ], metadata : Any ) {
107125 executor.pushIterator(dataIterator, Some (metadata), None )
108126 }
109127
110- /** Store the bytes of received data into Spark's memory. */
128+ /** Store the bytes of received data as a data block into Spark's memory. */
111129 def store (bytes : ByteBuffer ) {
112130 executor.pushBytes(bytes, None , None )
113131 }
114132
115- /** Store the bytes of received data into Spark's memory.
133+ /** Store the bytes of received data as a data block into Spark's memory.
116134 * The metadata will be associated with this block of data
117135 * for being used in the corresponding InputDStream.
118136 */
119- def store (bytes : ByteBuffer , metadata : Any = null ) {
137+ def store (bytes : ByteBuffer , metadata : Any ) {
120138 executor.pushBytes(bytes, Some (metadata), None )
121139 }
122140
@@ -143,26 +161,26 @@ abstract class NetworkReceiver[T](val storageLevel: StorageLevel) extends Serial
143161 * The delay is defined by the Spark configuration
144162 * `spark.streaming.receiverRestartDelay`.
145163 */
146- def restart (message : String , exception : Throwable ) {
147- executor.restartReceiver(message, exception )
164+ def restart (message : String , error : Throwable ) {
165+ executor.restartReceiver(message, Some (error) )
148166 }
149167
150168 /**
151169 * Restart the receiver. This will call `onStop()` immediately and return.
152170 * Asynchronously, after the given delay, `onStart()` will be called.
153171 */
154- def restart (message : String , throwable : Throwable , millisecond : Int ) {
155- executor.restartReceiver(message, throwable , millisecond)
172+ def restart (message : String , error : Throwable , millisecond : Int ) {
173+ executor.restartReceiver(message, Some (error) , millisecond)
156174 }
157175
158176 /** Stop the receiver completely. */
159177 def stop (message : String ) {
160- executor.stop(message)
178+ executor.stop(message, None )
161179 }
162180
163181 /** Stop the receiver completely due to an exception */
164- def stop (message : String , exception : Throwable ) {
165- executor.stop(message, exception )
182+ def stop (message : String , error : Throwable ) {
183+ executor.stop(message, Some (error) )
166184 }
167185
168186 def isStarted (): Boolean = {
@@ -175,7 +193,7 @@ abstract class NetworkReceiver[T](val storageLevel: StorageLevel) extends Serial
175193 }
176194
177195 /** Get unique identifier of this receiver. */
178- def receiverId = id
196+ def streamId = id
179197
180198 /*
181199 * =================
0 commit comments