Skip to content

Commit 91da0bb

Browse files
committed
Merge branch 'master' of https://github.com/apache/spark into SPARK-1609
2 parents 0640852 + 87cf35c commit 91da0bb

File tree

14 files changed

+307
-242
lines changed

14 files changed

+307
-242
lines changed

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 12 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -300,10 +300,17 @@ class SparkContext(config: SparkConf) extends Logging {
300300

301301
// Create and start the scheduler
302302
private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master)
303-
taskScheduler.start()
303+
@volatile private[spark] var dagScheduler: DAGScheduler = _
304+
try {
305+
dagScheduler = new DAGScheduler(this)
306+
} catch {
307+
case e: Exception => throw
308+
new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage))
309+
}
304310

305-
@volatile private[spark] var dagScheduler = new DAGScheduler(this)
306-
dagScheduler.start()
311+
// start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's
312+
// constructor
313+
taskScheduler.start()
307314

308315
private[spark] val cleaner: Option[ContextCleaner] = {
309316
if (conf.getBoolean("spark.cleaner.referenceTracking", true)) {
@@ -1022,8 +1029,8 @@ class SparkContext(config: SparkConf) extends Logging {
10221029
partitions: Seq[Int],
10231030
allowLocal: Boolean,
10241031
resultHandler: (Int, U) => Unit) {
1025-
partitions.foreach{ p =>
1026-
require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p")
1032+
if (dagScheduler == null) {
1033+
throw new SparkException("SparkContext has been shutdown")
10271034
}
10281035
val callSite = getCallSite
10291036
val cleanedFunc = clean(func)
@@ -1132,9 +1139,6 @@ class SparkContext(config: SparkConf) extends Logging {
11321139
resultHandler: (Int, U) => Unit,
11331140
resultFunc: => R): SimpleFutureAction[R] =
11341141
{
1135-
partitions.foreach{ p =>
1136-
require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p")
1137-
}
11381142
val cleanF = clean(processPartition)
11391143
val callSite = getCallSite
11401144
val waiter = dagScheduler.submitJob(

core/src/main/scala/org/apache/spark/rdd/RDD.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1142,9 +1142,9 @@ abstract class RDD[T: ClassTag](
11421142
@transient private var doCheckpointCalled = false
11431143

11441144
/**
1145-
* Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler
1146-
* after a job using this RDD has completed (therefore the RDD has been materialized and
1147-
* potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs.
1145+
* Performs the checkpointing of this RDD by saving this. It is called after a job using this RDD
1146+
* has completed (therefore the RDD has been materialized and potentially stored in memory).
1147+
* doCheckpoint() is called recursively on the parent RDDs.
11481148
*/
11491149
private[spark] def doCheckpoint() {
11501150
if (!doCheckpointCalled) {

0 commit comments

Comments
 (0)