/** Start the endpoint and receiver execution thread. */
def
start()
:
Unit
=
synchronized {
if
(isTrackerStarted) {
throw
new
SparkException(
"ReceiverTracker already started"
)
}
if
(!receiverInputStreams.isEmpty) {
endpoint
=
ssc.env.rpcEnv.setupEndpoint(
"ReceiverTracker"
,
new
ReceiverTrackerEndpoint(ssc.env.rpcEnv))
if
(!skipReceiverLaunch) launchReceivers()
logInfo(
"ReceiverTracker started"
)
trackerState
=
Started
}
}
private
def
launchReceivers()
:
Unit
=
{
val
receivers
=
receiverInputStreams.map(nis
=
> {
val
rcvr
=
nis.getReceiver()
rcvr.setReceiverId(nis.id)
rcvr
})
runDummySparkJob()
logInfo(
"Starting "
+ receivers.length +
" receivers"
)
endpoint.send(StartAllReceivers(receivers))
}
private
def
runDummySparkJob()
:
Unit
=
{
if
(!ssc.sparkContext.isLocal) {
ssc.sparkContext.makeRDD(
1
to
50
,
50
).map(x
=
> (x,
1
)).reduceByKey(
_
+
_
,
20
).collect()
}
assert(getExecutors.nonEmpty)
}
private
def
getExecutors
:
Seq[ExecutorCacheTaskLocation]
=
{
if
(ssc.sc.isLocal) {
val
blockManagerId
=
ssc.sparkContext.env.blockManager.blockManagerId
Seq(ExecutorCacheTaskLocation(blockManagerId.host, blockManagerId.executorId))
}
else
{
ssc.sparkContext.env.blockManager.master.getMemoryStatus.filter {
case
(blockManagerId,
_
)
=
>
blockManagerId.executorId !
=
SparkContext.DRIVER
_
IDENTIFIER
// Ignore the driver location
}.map {
case
(blockManagerId,
_
)
=
>
ExecutorCacheTaskLocation(blockManagerId.host, blockManagerId.executorId)
}.toSeq
}
}
case
StartAllReceivers(receivers)
=
>
val
scheduledLocations
=
schedulingPolicy.scheduleReceivers(receivers, getExecutors)
for
(receiver <- receivers) {
val
executors
=
scheduledLocations(receiver.streamId)
updateReceiverScheduledExecutors(receiver.streamId, executors)
receiverPreferredLocations(receiver.streamId)
=
receiver.preferredLocation
startReceiver(receiver, executors)
}
val
receiverRDD
:
RDD[Receiver[
_
]]
=
if
(scheduledLocations.isEmpty) {
ssc.sc.makeRDD(Seq(receiver),
1
)
}
else
{
val
preferredLocations
=
scheduledLocations.map(
_
.toString).distinct
ssc.sc.makeRDD(Seq(receiver -> preferredLocations))
}
val
future
=
ssc.sparkContext.submitJob[Receiver[
_
], Unit, Unit](
receiverRDD, startReceiverFunc, Seq(
0
), (
_
,
_
)
=
> Unit, ())
val
startReceiverFunc
:
Iterator[Receiver[
_
]]
=
> Unit
=
(iterator
:
Iterator[Receiver[
_
]])
=
> {
if
(!iterator.hasNext) {
throw
new
SparkException(
"Could not start receiver as object not found."
)
}
if
(TaskContext.get().attemptNumber()
==
0
) {
val
receiver
=
iterator.next()
assert(iterator.hasNext
==
false
)
val
supervisor
=
new
ReceiverSupervisorImpl(
receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption)
supervisor.start()
supervisor.awaitTermination()
}
else
{
// It's restarted by TaskScheduler, but we want to reschedule it again. So exit it.
}
}
override
protected
def
onReceiverStart()
:
Boolean
=
{
val
msg
=
RegisterReceiver(
streamId, receiver.getClass.getSimpleName, host, executorId, endpoint)
trackerEndpoint.askWithRetry[Boolean](msg)
}
def
startReceiver()
:
Unit
=
synchronized {
try
{
if
(onReceiverStart()) {
logInfo(
"Starting receiver"
)
receiverState
=
Started
receiver.onStart()
logInfo(
"Called receiver onStart"
)
}
else
{
// The driver refused us
stop(
"Registered unsuccessfully because Driver refused to start receiver "
+ streamId, None)
}
}
catch
{
case
NonFatal(t)
=
>
stop(
"Error starting receiver "
+ streamId, Some(t))
}
}
future.onComplete {
case
Success(
_
)
=
>
if
(!shouldStartReceiver) {
onReceiverJobFinish(receiverId)
}
else
{
logInfo(s
"Restarting Receiver $receiverId"
)
self.send(RestartReceiver(receiver))
}
case
Failure(e)
=
>
if
(!shouldStartReceiver) {
onReceiverJobFinish(receiverId)
}
else
{
logError(
"Receiver has been stopped. Try to restart it."
, e)
logInfo(s
"Restarting Receiver $receiverId"
)
self.send(RestartReceiver(receiver))
}
}(submitJobThreadPool)