-
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-2521] Broadcast RDD object (instead of sending it along with every task). #1452
Changes from 4 commits
04b17f0
754085f
c3b6f11
ade6eac
762e0be
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 |
---|---|---|
|
@@ -17,134 +17,68 @@ | |
|
||
package org.apache.spark.scheduler | ||
|
||
import scala.language.existentials | ||
import java.nio.ByteBuffer | ||
|
||
import java.io._ | ||
import java.util.zip.{GZIPInputStream, GZIPOutputStream} | ||
|
||
import scala.collection.mutable.HashMap | ||
|
||
import org.apache.spark._ | ||
import org.apache.spark.rdd.{RDD, RDDCheckpointData} | ||
|
||
private[spark] object ResultTask { | ||
|
||
// A simple map between the stage id to the serialized byte array of a task. | ||
// Served as a cache for task serialization because serialization can be | ||
// expensive on the master node if it needs to launch thousands of tasks. | ||
private val serializedInfoCache = new HashMap[Int, Array[Byte]] | ||
|
||
def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = | ||
{ | ||
synchronized { | ||
val old = serializedInfoCache.get(stageId).orNull | ||
if (old != null) { | ||
old | ||
} else { | ||
val out = new ByteArrayOutputStream | ||
val ser = SparkEnv.get.closureSerializer.newInstance() | ||
val objOut = ser.serializeStream(new GZIPOutputStream(out)) | ||
objOut.writeObject(rdd) | ||
objOut.writeObject(func) | ||
objOut.close() | ||
val bytes = out.toByteArray | ||
serializedInfoCache.put(stageId, bytes) | ||
bytes | ||
} | ||
} | ||
} | ||
|
||
def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = | ||
{ | ||
val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) | ||
val ser = SparkEnv.get.closureSerializer.newInstance() | ||
val objIn = ser.deserializeStream(in) | ||
val rdd = objIn.readObject().asInstanceOf[RDD[_]] | ||
val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _] | ||
(rdd, func) | ||
} | ||
|
||
def removeStage(stageId: Int) { | ||
serializedInfoCache.remove(stageId) | ||
} | ||
|
||
def clearCache() { | ||
synchronized { | ||
serializedInfoCache.clear() | ||
} | ||
} | ||
} | ||
|
||
import org.apache.spark.broadcast.Broadcast | ||
import org.apache.spark.rdd.RDD | ||
|
||
/** | ||
* A task that sends back the output to the driver application. | ||
* | ||
* See [[org.apache.spark.scheduler.Task]] for more information. | ||
* See [[Task]] for more information. | ||
* | ||
* @param stageId id of the stage this task belongs to | ||
* @param rdd input to func | ||
* @param rddBinary broadcast version of of the serialized RDD | ||
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. *of - ha! 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 past tense -- broadcasted |
||
* @param func a function to apply on a partition of the RDD | ||
* @param _partitionId index of the number in the RDD | ||
* @param partition partition of the RDD this task is associated with | ||
* @param locs preferred task execution locations for locality scheduling | ||
* @param outputId index of the task in this job (a job can launch tasks on only a subset of the | ||
* input RDD's partitions). | ||
*/ | ||
private[spark] class ResultTask[T, U]( | ||
stageId: Int, | ||
var rdd: RDD[T], | ||
var func: (TaskContext, Iterator[T]) => U, | ||
_partitionId: Int, | ||
val rddBinary: Broadcast[Array[Byte]], | ||
val func: (TaskContext, Iterator[T]) => U, | ||
val partition: Partition, | ||
@transient locs: Seq[TaskLocation], | ||
var outputId: Int) | ||
extends Task[U](stageId, _partitionId) with Externalizable { | ||
|
||
def this() = this(0, null, null, 0, null, 0) | ||
|
||
var split = if (rdd == null) null else rdd.partitions(partitionId) | ||
val outputId: Int) | ||
extends Task[U](stageId, partition.index) with Serializable { | ||
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 partitionId the same thing as partition.index? 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. @mateiz and I looked and it seems so. |
||
|
||
// TODO: Should we also broadcast func? For that we would need a place to | ||
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. Perhaps we can just turn this into a JIRA rather than keeping it here in the code. |
||
// keep a reference to it (perhaps in DAGScheduler's job object). | ||
|
||
def this( | ||
stageId: Int, | ||
rdd: RDD[T], | ||
func: (TaskContext, Iterator[T]) => U, | ||
partitionId: Int, | ||
locs: Seq[TaskLocation], | ||
outputId: Int) = { | ||
this(stageId, rdd.broadcasted, func, rdd.partitions(partitionId), locs, outputId) | ||
} | ||
|
||
@transient private val preferredLocs: Seq[TaskLocation] = { | ||
@transient private[this] val preferredLocs: Seq[TaskLocation] = { | ||
if (locs == null) Nil else locs.toSet.toSeq | ||
} | ||
|
||
override def runTask(context: TaskContext): U = { | ||
// Deserialize the RDD using the broadcast variable. | ||
val ser = SparkEnv.get.closureSerializer.newInstance() | ||
val rdd = ser.deserialize[RDD[T]](ByteBuffer.wrap(rddBinary.value), | ||
Thread.currentThread.getContextClassLoader) | ||
metrics = Some(context.taskMetrics) | ||
try { | ||
func(context, rdd.iterator(split, context)) | ||
func(context, rdd.iterator(partition, context)) | ||
} finally { | ||
context.executeOnCompleteCallbacks() | ||
} | ||
} | ||
|
||
// This is only callable on the driver side. | ||
override def preferredLocations: Seq[TaskLocation] = preferredLocs | ||
|
||
override def toString = "ResultTask(" + stageId + ", " + partitionId + ")" | ||
|
||
override def writeExternal(out: ObjectOutput) { | ||
RDDCheckpointData.synchronized { | ||
split = rdd.partitions(partitionId) | ||
out.writeInt(stageId) | ||
val bytes = ResultTask.serializeInfo( | ||
stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _]) | ||
out.writeInt(bytes.length) | ||
out.write(bytes) | ||
out.writeInt(partitionId) | ||
out.writeInt(outputId) | ||
out.writeLong(epoch) | ||
out.writeObject(split) | ||
} | ||
} | ||
|
||
override def readExternal(in: ObjectInput) { | ||
val stageId = in.readInt() | ||
val numBytes = in.readInt() | ||
val bytes = new Array[Byte](numBytes) | ||
in.readFully(bytes) | ||
val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes) | ||
rdd = rdd_.asInstanceOf[RDD[T]] | ||
func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] | ||
partitionId = in.readInt() | ||
outputId = in.readInt() | ||
epoch = in.readLong() | ||
split = in.readObject().asInstanceOf[Partition] | ||
} | ||
} |
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.
It would be nice to add this in this patch, we can just choose a threshold