-
Notifications
You must be signed in to change notification settings - Fork 385
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
Support Cancellation of Spark Jobs #665
Changes from all commits
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,26 @@ | ||
package spark | ||
|
||
trait InterruptibleIterator[+T] extends Iterator[T]{ | ||
|
||
override def hasNext(): Boolean = { | ||
if (!Thread.currentThread().isInterrupted()) { | ||
true | ||
} else { | ||
throw new InterruptedException ("Thread interrupted during RDD iteration") | ||
} | ||
} | ||
|
||
} | ||
|
||
class InterruptibleIteratorDecorator[T](delegate: Iterator[T]) | ||
extends AnyRef with InterruptibleIterator[T] { | ||
|
||
override def hasNext(): Boolean = { | ||
super.hasNext | ||
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. It seems kinda weird to me that 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. Yes, this seems odd to me, too. Is there some reason why |
||
delegate.hasNext | ||
} | ||
|
||
override def next(): T = { | ||
delegate.next() | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2,16 +2,16 @@ package spark.executor | |
|
||
import java.io.{File, FileOutputStream} | ||
import java.net.{URI, URL, URLClassLoader} | ||
import java.nio.ByteBuffer | ||
import java.util.concurrent._ | ||
|
||
import scala.collection.JavaConversions._ | ||
import scala.collection.mutable.{ArrayBuffer,ConcurrentMap, HashMap, Map} | ||
import scala.concurrent.JavaConversions._ | ||
import org.apache.hadoop.fs.FileUtil | ||
|
||
import scala.collection.mutable.{ArrayBuffer, Map, HashMap} | ||
|
||
import spark.broadcast._ | ||
import spark.scheduler._ | ||
import spark._ | ||
import java.nio.ByteBuffer | ||
import spark.scheduler.cluster.TaskDescription | ||
|
||
/** | ||
* The Mesos executor for Spark. | ||
|
@@ -79,14 +79,26 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert | |
val threadPool = new ThreadPoolExecutor( | ||
1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable]) | ||
|
||
val tasks: ConcurrentMap[Long, FutureTask[_]] = new ConcurrentHashMap[Long, FutureTask[_]]() | ||
def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) { | ||
threadPool.execute(new TaskRunner(context, taskId, serializedTask)) | ||
val runner = new TaskRunner(context, taskId, serializedTask) | ||
val task = threadPool.submit(runner).asInstanceOf[FutureTask[_]] | ||
tasks.put(taskId, task) | ||
|
||
} | ||
|
||
def killTask(context: ExecutorBackend, taskId: Long, executorId: String) { | ||
val task = tasks.get(taskId) | ||
task match { | ||
case Some(t) => t.cancel(true) | ||
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. It may be good to do tasks.remove(taskId) here too in case the task thread had never started and so we never got to the "finally" in its run() method. 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. good point, will do thanks |
||
case None => | ||
} | ||
} | ||
|
||
class TaskRunner(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) | ||
extends Runnable { | ||
|
||
override def run() { | ||
override def run(): Unit = { | ||
val startTime = System.currentTimeMillis() | ||
SparkEnv.set(env) | ||
Thread.currentThread.setContextClassLoader(replClassLoader) | ||
|
@@ -138,6 +150,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert | |
logError("Exception in task ID " + taskId, t) | ||
//System.exit(1) | ||
} | ||
} finally { | ||
tasks.remove(taskId) | ||
} | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -278,6 +278,41 @@ class DAGScheduler( | |
return listener.awaitResult() // Will throw an exception if the job fails | ||
} | ||
|
||
def killJob(jobId: Int, reason: String) | ||
{ | ||
logInfo("Killing Job %s".format(jobId)) | ||
val j = activeJobs.find(j => j.runId.equals(jobId)) | ||
j match { | ||
case Some(job) => killJob(job, reason) | ||
case None => Unit | ||
} | ||
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.
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. nice, thanks...did not know this pattern 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. http://ymasory.github.io/error-handling-in-scala/#slide-52 Not everything that Yuvi presents there can be used in pre-2.10 Scala. 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. Oh, and in this particular case where |
||
} | ||
|
||
private def killJob(job: ActiveJob, reason: String) { | ||
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. 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. Yes, that is actually true. To do this properly we'll need to do some kind of reference-counting on the stages (keep a list of which jobs currently want to run this stage). One difference here is that killJob is called by the user and for the first use case, of Shark, it's probably going to be fine. But it would be good to either track this properly or send a warning. 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. That's pretty much the conclusion that I was arriving at. I'll work on the reference-counting refactoring. Should be doable independently of this PR and only require a minimal change here once it is done. 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. Cool, that would be great to have. |
||
logInfo("Killing Job and cleaning up stages %s".format(job.runId)) | ||
activeJobs.remove(job) | ||
idToActiveJob.remove(job.runId) | ||
val stage = job.finalStage | ||
resultStageToJob.remove(stage) | ||
killStage(stage) | ||
// recursively remove all parent stages | ||
stage.parents.foreach(p => killStage(p)) | ||
job.listener.jobFailed(new SparkException("Job failed: " + reason)) | ||
} | ||
|
||
private def killStage(stage: Stage) { | ||
logInfo("Killing Stage %s".format(stage.id)) | ||
idToStage.remove(stage.id) | ||
if (stage.isShuffleMap) { | ||
shuffleToMapStage.remove(stage.id) | ||
} | ||
waiting.remove(stage) | ||
pendingTasks.remove(stage) | ||
running.remove(stage) | ||
taskSched.killTasks(stage.id) | ||
stage.parents.foreach(p => killStage(p)) | ||
} | ||
|
||
/** | ||
* Process one event retrieved from the event queue. | ||
* Returns true if we should stop the event loop. | ||
|
@@ -495,7 +530,11 @@ class DAGScheduler( | |
*/ | ||
private def handleTaskCompletion(event: CompletionEvent) { | ||
val task = event.task | ||
val stage = idToStage(task.stageId) | ||
val stageId = task.stageId | ||
if (!idToStage.contains(stageId)) { | ||
return; | ||
} | ||
val stage = idToStage(stageId) | ||
|
||
def markStageAsFinished(stage: Stage) = { | ||
val serviceTime = stage.submissionTime match { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,23 +1,48 @@ | ||
package spark.scheduler | ||
|
||
import spark.serializer.SerializerInstance | ||
import java.io.{DataInputStream, DataOutputStream} | ||
import java.nio.ByteBuffer | ||
import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream | ||
import spark.util.ByteBufferInputStream | ||
import java.util.concurrent.{Callable, ExecutionException, Future, FutureTask} | ||
import scala.collection.mutable.HashMap | ||
import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream | ||
import spark.executor.TaskMetrics | ||
import spark.serializer.SerializerInstance | ||
import spark.util.ByteBufferInputStream | ||
|
||
|
||
/** | ||
* A task to execute on a worker node. | ||
*/ | ||
private[spark] abstract class Task[T](val stageId: Int) extends Serializable { | ||
def run(attemptId: Long): T | ||
@volatile @transient var f: FutureTask[T] = null | ||
|
||
def run(attemptId: Long): T = { | ||
f = new FutureTask(new Callable[T] { | ||
def call(): T = { | ||
runInterruptibly(attemptId) | ||
} | ||
}) | ||
try { | ||
f.run() | ||
f.get() | ||
} catch { | ||
case e: Exception => throw e.getCause() | ||
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. Should we be catching Throwable here? 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. yes, you are right., we should be catching throwable here. |
||
} | ||
} | ||
|
||
def runInterruptibly(attemptId: Long): T | ||
|
||
def preferredLocations: Seq[String] = Nil | ||
|
||
var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. | ||
|
||
var metrics: Option[TaskMetrics] = None | ||
|
||
def kill(): Unit = { | ||
if (f != null) { | ||
f.cancel(true) | ||
} | ||
} | ||
|
||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,6 +22,7 @@ import java.util.Properties | |
private[spark] trait SchedulableBuilder { | ||
def buildPools() | ||
def addTaskSetManager(manager: Schedulable, properties: Properties) | ||
def popTaskSetManagers(stageId: Int): Iterable[Schedulable] | ||
} | ||
|
||
private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { | ||
|
@@ -33,6 +34,16 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends Schedula | |
override def addTaskSetManager(manager: Schedulable, properties: Properties) { | ||
rootPool.addSchedulable(manager) | ||
} | ||
|
||
override def popTaskSetManagers(stageId: Int) = { | ||
val s = rootPool.schedulableNameToSchedulable.values.filter { | ||
_.stageId == stageId | ||
} | ||
s.foreach { | ||
rootPool.removeSchedulable(_) | ||
} | ||
s | ||
} | ||
} | ||
|
||
private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { | ||
|
@@ -112,4 +123,14 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula | |
parentPool.addSchedulable(manager) | ||
logInfo("Added task set " + manager.name + " tasks to pool "+poolName) | ||
} | ||
|
||
override def popTaskSetManagers(stageId: Int) = { | ||
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. can you avoid repeating this method, eg. move to a base class? 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. good point . thanks will do |
||
val s = rootPool.schedulableNameToSchedulable.values.filter { | ||
_.stageId == stageId | ||
} | ||
s.foreach { | ||
rootPool.removeSchedulable(_) | ||
} | ||
s | ||
} | ||
} |
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.
excess indentation