-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
SPARK-2099. Report progress while task is running. #1056
Changes from all commits
0dae734
3bda974
3084f10
51fa396
38dffde
132aec7
93b9fdb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,46 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark | ||
|
||
import akka.actor.Actor | ||
import org.apache.spark.executor.TaskMetrics | ||
import org.apache.spark.storage.BlockManagerId | ||
import org.apache.spark.scheduler.TaskScheduler | ||
|
||
/** | ||
* A heartbeat from executors to the driver. This is a shared message used by several internal | ||
* components to convey liveness or execution information for in-progress tasks. | ||
*/ | ||
private[spark] case class Heartbeat( | ||
executorId: String, | ||
taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics | ||
blockManagerId: BlockManagerId) | ||
|
||
private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) | ||
|
||
/** | ||
* Lives in the driver to receive heartbeats from executors.. | ||
*/ | ||
private[spark] class HeartbeatReceiver(scheduler: TaskScheduler) extends Actor { | ||
override def receive = { | ||
case Heartbeat(executorId, taskMetrics, blockManagerId) => | ||
val response = HeartbeatResponse( | ||
!scheduler.executorHeartbeatReceived(executorId, taskMetrics, blockManagerId)) | ||
sender ! response | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -32,6 +32,9 @@ import org.apache.spark._ | |
import org.apache.spark.TaskState.TaskState | ||
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode | ||
import org.apache.spark.util.Utils | ||
import org.apache.spark.executor.TaskMetrics | ||
import org.apache.spark.storage.BlockManagerId | ||
import akka.actor.Props | ||
|
||
/** | ||
* Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. | ||
|
@@ -320,6 +323,26 @@ private[spark] class TaskSchedulerImpl( | |
} | ||
} | ||
|
||
/** | ||
* Update metrics for in-progress tasks and let the master know that the BlockManager is still | ||
* alive. Return true if the driver knows about the given block manager. Otherwise, return false, | ||
* indicating that the block manager should re-register. | ||
*/ | ||
override def executorHeartbeatReceived( | ||
execId: String, | ||
taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics | ||
blockManagerId: BlockManagerId): Boolean = { | ||
val metricsWithStageIds = taskMetrics.flatMap { | ||
case (id, metrics) => { | ||
taskIdToTaskSetId.get(id) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think there is an unlikely race here where (a) a task heartbeat gets enqueued to be sent (b) the task actually finishes and that message is sent, then taskIdToTaskSetId is cleared (c) the heartbeat arrives. This is possible since the heartbeater and the task execution itself are in different threads. Then you'd get an NPE here. Though extremely unlikely, it might be good to just log a warning and pass if the task set is not found. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. As per our discussion, this is using options everywhere, so it's good. |
||
.flatMap(activeTaskSets.get) | ||
.map(_.stageId) | ||
.map(x => (id, x, metrics)) | ||
} | ||
} | ||
dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) | ||
} | ||
|
||
def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { | ||
taskSetManager.handleTaskGettingResult(tid) | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
use Utils.namedThreadFactory or otherwise generate a named thread
okay, I now see that you do that further down. Utils.namedThreadFactory still looks like the best option.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not able to find Utils.namedThreadFactory, but my understanding is that named thread factories are used when using a thread pool and the threads can't be named directly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
#1604
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
And yes, it goes outside the scope of this PR, but one of the things that bugs me about the Spark code is how frequently we directly create threads with
new Thread
instead of with some standardized factory and perhaps with better defined and utilized thread pools.