-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-1141] [WIP] Parallelize Task Serialization #214
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
Changes from all commits
5379596
0bb3744
177195d
dee0fb9
fe0cd9d
48fe9f2
dae1dc3
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 |
|---|---|---|
|
|
@@ -29,9 +29,12 @@ import org.apache.mesos.{Scheduler => MScheduler} | |
| import org.apache.mesos._ | ||
| import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} | ||
|
|
||
| import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} | ||
| import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} | ||
| import org.apache.spark.{SparkEnv, Logging, SparkContext, SparkException, TaskState} | ||
| import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost} | ||
| import org.apache.spark.scheduler.{TaskDescWithoutSerializedTask, TaskDescription, TaskSchedulerImpl, WorkerOffer} | ||
| import org.apache.spark.util.Utils | ||
| import org.apache.spark.serializer.SerializerInstance | ||
|
|
||
|
|
||
| /** | ||
| * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a | ||
|
|
@@ -62,6 +65,30 @@ private[spark] class MesosSchedulerBackend( | |
|
|
||
| var classLoader: ClassLoader = null | ||
|
|
||
| val serializeWorkerPool = Utils.newDaemonFixedThreadPool( | ||
| scheduler.sc.conf.getInt("spark.scheduler.task.serialize.threads", 4), "task-serialization") | ||
|
|
||
| val env = SparkEnv.get | ||
| protected val serializer = new ThreadLocal[SerializerInstance] { | ||
| override def initialValue(): SerializerInstance = { | ||
| env.closureSerializer.newInstance() | ||
| } | ||
| } | ||
|
|
||
| class TaskMesosSerializedRunner(taskNoSer: TaskDescWithoutSerializedTask, | ||
|
Contributor
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. Is it possible to merge TaskMesosSerializedRunner and TaskCGSerializedRunner in the same class, or at least declare them in the same source file, so that we don't need to walk through multiple files in future updates?
Contributor
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 just noticed that @mridulm has proposed this in his last comment, I think it will be great if we can do in that way
Contributor
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. also, I have some hesitations about the class name...serializedRunner....any better name? not a native speaker....@kayousterhout @mridulm ? |
||
| taskList: JList[MesosTaskInfo], | ||
| slaveId: String, | ||
| scheduler: TaskSchedulerImpl) | ||
| extends Runnable { | ||
| override def run() { | ||
| // Serialize and return the task | ||
| val task = Utils.serializeTask(taskNoSer, scheduler.sc, serializer.get()) | ||
| taskList.synchronized { | ||
| taskList.add(createMesosTask(task, slaveId)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| override def start() { | ||
| synchronized { | ||
| classLoader = Thread.currentThread.getContextClassLoader | ||
|
|
@@ -213,9 +240,11 @@ private[spark] class MesosSchedulerBackend( | |
| val slaveId = offers(offerNum).getSlaveId.getValue | ||
| slaveIdsWithExecutors += slaveId | ||
| mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) | ||
| for (taskDesc <- taskList) { | ||
| taskIdToSlaveId(taskDesc.taskId) = slaveId | ||
| mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) | ||
| for (taskNoSer <- taskList) { | ||
| taskIdToSlaveId(taskNoSer.taskId) = slaveId | ||
| val taskList = mesosTasks(offerNum) | ||
| serializeWorkerPool.execute( | ||
| new TaskMesosSerializedRunner(taskNoSer, taskList, slaveId, scheduler)) | ||
| } | ||
| } | ||
| } | ||
|
|
@@ -297,6 +326,9 @@ private[spark] class MesosSchedulerBackend( | |
| } | ||
|
|
||
| override def stop() { | ||
| if (serializeWorkerPool != null) { | ||
| serializeWorkerPool.shutdownNow() | ||
| } | ||
| if (driver != null) { | ||
| driver.stop() | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,6 +25,7 @@ import org.apache.spark.{Logging, SparkEnv, TaskState} | |
| import org.apache.spark.TaskState.TaskState | ||
| import org.apache.spark.executor.{Executor, ExecutorBackend} | ||
| import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| private case class ReviveOffers() | ||
|
|
||
|
|
@@ -46,6 +47,7 @@ private[spark] class LocalActor( | |
|
|
||
| private val localExecutorId = "localhost" | ||
| private val localExecutorHostname = "localhost" | ||
| val ser = scheduler.sc.env.closureSerializer.newInstance() | ||
|
Contributor
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. en...can we just make it as private? |
||
|
|
||
| val executor = new Executor( | ||
| localExecutorId, localExecutorHostname, scheduler.conf.getAll, isLocal = true) | ||
|
|
@@ -67,8 +69,9 @@ private[spark] class LocalActor( | |
|
|
||
| def reviveOffers() { | ||
| val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) | ||
| for (task <- scheduler.resourceOffers(offers).flatten) { | ||
| for (taskNoSer <- scheduler.resourceOffers(offers).flatten) { | ||
| freeCores -= 1 | ||
| val task = Utils.serializeTask(taskNoSer, scheduler.sc, ser) | ||
| executor.launchTask(executorBackend, task.taskId, task.serializedTask) | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -34,9 +34,11 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder | |
| import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} | ||
| import org.json4s._ | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} | ||
| import org.apache.spark.scheduler.{TaskDescWithoutSerializedTask, TaskDescription, Task} | ||
|
|
||
| /** | ||
| * Various utility methods used by Spark. | ||
|
|
@@ -149,6 +151,21 @@ private[spark] object Utils extends Logging { | |
| buf | ||
| } | ||
|
|
||
| def serializeTask(taskNoSer: TaskDescWithoutSerializedTask, sc: SparkContext, | ||
| serializer: SerializerInstance) : TaskDescription = { | ||
| val startTime = System.currentTimeMillis() | ||
| // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here | ||
|
Contributor
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. is it difficult to capture the non-serializable objects here? as @kayousterhout said in one of my PRs, DAGScheduler knows too much about task-level details now...maybe we can do something to make the situation better.....
Contributor
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 checked the DAGScheduler code, L793, I think it's pretty straightforward to move that functionality to here, as you will have an unified implementation of SerializerAndRunner for all scheduler backend....https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L770, that line is actually just for checking if the task is serializable...... |
||
| // we assume the task can be serialized without exceptions. | ||
| val serializedTask = Task.serializeWithDependencies( | ||
| taskNoSer.taskObject, sc.addedFiles, sc.addedJars, serializer) | ||
| val timeTaken = System.currentTimeMillis() - startTime | ||
| logInfo("Serialized task %s as %d bytes in %d ms".format( | ||
| taskNoSer.taskName, serializedTask.limit, timeTaken)) | ||
| val task = new TaskDescription(taskNoSer.taskId, taskNoSer.executorId, | ||
| taskNoSer.taskName, taskNoSer.index, serializedTask) | ||
| task | ||
| } | ||
|
|
||
| private val shutdownDeletePaths = new scala.collection.mutable.HashSet[String]() | ||
|
|
||
| // Register the path to be deleted via shutdown hook | ||
|
|
||
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.
extra line