Skip to content

Commit a75c7a6

Browse files
committed
Address some PR comments and fixed other issues.
1 parent 91bfa72 commit a75c7a6

File tree

10 files changed

+147
-121
lines changed

10 files changed

+147
-121
lines changed

external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ public class JavaFlumeStreamSuite extends LocalJavaStreamingContext {
2929
public void testFlumeStream() {
3030
// tests the API, does not actually test data receiving
3131
JavaNetworkInputDStream<SparkFlumeEvent> test1 = FlumeUtils.createStream(ssc, "localhost", 12345);
32-
JavaNetworkInputDStream<SparkFlumeEvent> test2 = FlumeUtils.createStream(ssc, "localhost", 12345,
32+
JavaNetworkInputDStream<SparkFlumeEvent> test2 = FlumeUtils.createStream(ssc, "localhost", 12345,
3333
StorageLevel.MEMORY_AND_DISK_SER_2());
3434
}
3535
}

external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ public void testKafkaStream() {
3535
// tests the API, does not actually test data receiving
3636
JavaPairNetworkInputDStream<String, String> test1 =
3737
KafkaUtils.createStream(ssc, "localhost:12345", "group", topics);
38-
JavaPairNetworkInputDStream<String, String> test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics,
38+
JavaPairNetworkInputDStream<String, String> test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics,
3939
StorageLevel.MEMORY_AND_DISK_SER_2());
4040

4141
HashMap<String, String> kafkaParams = Maps.newHashMap();

streaming/src/main/scala/org/apache/spark/streaming/receiver/NetworkReceiver.scala

Lines changed: 46 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark.streaming.receiver
2020
import java.nio.ByteBuffer
2121

2222
import scala.collection.mutable.ArrayBuffer
23+
import scala.collection.JavaConversions._
2324

2425
import 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
*/
5154
abstract 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
* =================

streaming/src/main/scala/org/apache/spark/streaming/receiver/NetworkReceiverExecutor.scala

Lines changed: 34 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,6 @@ private[streaming] abstract class NetworkReceiverExecutor(
3636
conf: SparkConf
3737
) extends Logging {
3838

39-
4039
/** Enumeration to identify current state of the StreamingContext */
4140
object NetworkReceiverState extends Enumeration {
4241
type CheckpointState = Value
@@ -48,41 +47,38 @@ private[streaming] abstract class NetworkReceiverExecutor(
4847
receiver.attachExecutor(this)
4948

5049
/** Receiver id */
51-
protected val receiverId = receiver.receiverId
52-
53-
/** Message associated with the stopping of the receiver */
54-
protected var stopMessage = ""
55-
56-
/** Exception associated with the stopping of the receiver */
57-
protected var stopException: Throwable = null
50+
protected val streamId = receiver.streamId
5851

5952
/** Has the receiver been marked for stop. */
6053
private val stopLatch = new CountDownLatch(1)
6154

62-
/** Time between a receiver is stopped */
63-
private val restartDelay = conf.getInt("spark.streaming.receiverRestartDelay", 2000)
55+
/** Time between a receiver is stopped and started again */
56+
private val defaultRestartDelay = conf.getInt("spark.streaming.receiverRestartDelay", 2000)
57+
58+
/** Exception associated with the stopping of the receiver */
59+
@volatile protected var stoppingError: Throwable = null
6460

6561
/** State of the receiver */
66-
private[streaming] var receiverState = Initialized
62+
@volatile private[streaming] var receiverState = Initialized
6763

6864
/** Push a single data item to backend data store. */
6965
def pushSingle(data: Any)
7066

71-
/** Push a byte buffer to backend data store. */
67+
/** Store the bytes of received data as a data block into Spark's memory. */
7268
def pushBytes(
7369
bytes: ByteBuffer,
7470
optionalMetadata: Option[Any],
7571
optionalBlockId: Option[StreamBlockId]
7672
)
7773

78-
/** Push an iterator of objects as a block to backend data store. */
74+
/** Store a iterator of received data as a data block into Spark's memory. */
7975
def pushIterator(
8076
iterator: Iterator[_],
8177
optionalMetadata: Option[Any],
8278
optionalBlockId: Option[StreamBlockId]
8379
)
8480

85-
/** Push an ArrayBuffer of object as a block to back data store. */
81+
/** Store an ArrayBuffer of received data as a data block into Spark's memory. */
8682
def pushArrayBuffer(
8783
arrayBuffer: ArrayBuffer[_],
8884
optionalMetadata: Option[Any],
@@ -97,57 +93,46 @@ private[streaming] abstract class NetworkReceiverExecutor(
9793
startReceiver()
9894
}
9995

100-
/**
101-
* Mark the executor and the receiver for stopping
102-
*/
103-
def stop(message: String, exception: Throwable = null) {
104-
stopMessage = message
105-
stopException = exception
106-
stopReceiver()
96+
/** Mark the executor and the receiver for stopping */
97+
def stop(message: String, error: Option[Throwable]) {
98+
stoppingError = error.orNull
99+
stopReceiver(message, error)
107100
stopLatch.countDown()
108-
if (exception != null) {
109-
logError("Stopped executor: " + message, exception)
110-
} else {
111-
logWarning("Stopped executor: " + message)
112-
}
113101
}
114102

115103
/** Start receiver */
116104
def startReceiver(): Unit = synchronized {
117105
try {
118106
logInfo("Starting receiver")
119-
stopMessage = ""
120-
stopException = null
121107
onReceiverStart()
122108
receiverState = Started
123109
} catch {
124110
case t: Throwable =>
125-
stop("Error starting receiver " + receiverId, t)
111+
stop("Error starting receiver " + streamId, Some(t))
126112
}
127113
}
128114

129115
/** Stop receiver */
130-
def stopReceiver(): Unit = synchronized {
116+
def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized {
131117
try {
132118
receiverState = Stopped
133-
onReceiverStop()
119+
onReceiverStop(message, error)
134120
} catch {
135121
case t: Throwable =>
136-
stop("Error stopping receiver " + receiverId, t)
122+
stop("Error stopping receiver " + streamId, Some(t))
137123
}
138124
}
139125

140126
/** Restart receiver with delay */
141-
def restartReceiver(message: String, throwable: Throwable = null) {
142-
val defaultRestartDelay = conf.getInt("spark.streaming.receiverRestartDelay", 2000)
143-
restartReceiver(message, throwable, defaultRestartDelay)
127+
def restartReceiver(message: String, error: Option[Throwable] = None) {
128+
restartReceiver(message, error, defaultRestartDelay)
144129
}
145130

146131
/** Restart receiver with delay */
147-
def restartReceiver(message: String, exception: Throwable, delay: Int) {
148-
logWarning("Restarting receiver with delay " + delay + " ms: " + message, exception)
149-
reportError(message, exception)
150-
stopReceiver()
132+
def restartReceiver(message: String, error: Option[Throwable], delay: Int) {
133+
logWarning("Restarting receiver with delay " + delay + " ms: " + message,
134+
error.getOrElse(null))
135+
stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error)
151136
future {
152137
logDebug("Sleeping for " + delay)
153138
Thread.sleep(delay)
@@ -166,25 +151,30 @@ private[streaming] abstract class NetworkReceiverExecutor(
166151
}
167152

168153
/** Called when the receiver needs to be stopped */
169-
protected def onReceiverStop(): Unit = synchronized {
154+
protected def onReceiverStop(message: String, error: Option[Throwable]): Unit = synchronized {
170155
// Call user-defined onStop()
171156
logInfo("Calling receiver onStop")
172157
receiver.onStop()
173158
logInfo("Called receiver onStop")
174159
}
175160

176161
/** Check if receiver has been marked for stopping */
177-
def isReceiverStarted() = synchronized {
162+
def isReceiverStarted() = {
178163
logDebug("state = " + receiverState)
179164
receiverState == Started
180165
}
181166

182167
/** Wait the thread until the executor is stopped */
183-
def awaitStop() {
168+
def awaitTermination() {
184169
stopLatch.await()
185170
logInfo("Waiting for executor stop is over")
186-
if (stopException != null) {
187-
throw new Exception(stopMessage, stopException)
171+
if (stoppingError != null) {
172+
logError("Stopped executor with error: " + stoppingError)
173+
} else {
174+
logWarning("Stopped executor without error")
175+
}
176+
if (stoppingError != null) {
177+
throw stoppingError
188178
}
189179
}
190180
}

0 commit comments

Comments
 (0)