-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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-1712]: TaskDescription instance is too big causes Spark to hang #694
Closed
Closed
Changes from 25 commits
Commits
Show all changes
26 commits
Select commit
Hold shift + click to select a range
2a89adc
SPARK-1712: TaskDescription instance is too big causes Spark to hang
witgo 86e2048
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 743a7ad
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 3ea1ca1
remove duplicate serialize
witgo 0e29eac
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 7965580
fix Statement order
witgo 1d35c7d
fix hang
witgo 9e4ffa7
review commit
witgo 4afe71d
review commit
witgo 158b2dc
review commit
witgo 0a428cf
add unit tests
witgo 062c182
fix small bug for code style
witgo 1d35c3c
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 689495d
fix scala style bug
witgo f76679b
merge master
witgo b1174bd
merge master
witgo b0930b0
review commit
witgo 03cc562
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 9a5cfad
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 926bd6a
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 3b6d48c
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 44a59ee
review commit
witgo 63636b6
review commit
witgo 52e6752
reset test SparkContext
witgo 83ce29b
Merge branch 'master' of https://github.com/apache/spark into SPARK-1…
witgo 0f52483
review commit
witgo File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,10 +27,10 @@ import akka.actor._ | |
import akka.pattern.ask | ||
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} | ||
|
||
import org.apache.spark.{Logging, SparkException, TaskState} | ||
import org.apache.spark.{SparkEnv, Logging, SparkException, TaskState} | ||
import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} | ||
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ | ||
import org.apache.spark.util.{AkkaUtils, Utils} | ||
import org.apache.spark.util.{SerializableBuffer, AkkaUtils, Utils} | ||
|
||
/** | ||
* A scheduler backend that waits for coarse grained executors to connect to it through Akka. | ||
|
@@ -48,6 +48,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A | |
var totalCoreCount = new AtomicInteger(0) | ||
val conf = scheduler.sc.conf | ||
private val timeout = AkkaUtils.askTimeout(conf) | ||
private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) | ||
|
||
class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { | ||
private val executorActor = new HashMap[String, ActorRef] | ||
|
@@ -140,8 +141,29 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A | |
// Launch tasks returned by a set of resource offers | ||
def launchTasks(tasks: Seq[Seq[TaskDescription]]) { | ||
for (task <- tasks.flatten) { | ||
freeCores(task.executorId) -= scheduler.CPUS_PER_TASK | ||
executorActor(task.executorId) ! LaunchTask(task) | ||
val ser = SparkEnv.get.closureSerializer.newInstance() | ||
val serializedTask = ser.serialize(task) | ||
if (serializedTask.limit >= akkaFrameSize - 1024) { | ||
val taskSetId = scheduler.taskIdToTaskSetId(task.taskId) | ||
scheduler.activeTaskSets.get(taskSetId).foreach { taskSet => | ||
try { | ||
var msg = "Serialized task %s:%d was %d bytes which " + | ||
"exceeds spark.akka.frameSize (%d bytes). " + | ||
"Consider using broadcast variables for large values." | ||
msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize) | ||
taskSet.abort(msg) | ||
} catch { | ||
case e: Exception => logError("Exception in error callback", e) | ||
} | ||
} | ||
// scheduler.error(msg) | ||
// TODO: Need to throw an exception? | ||
// throw new SparkException(msg) | ||
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. Do we still need this code in here if we're aborting the TaskSet? 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, it should be removed. |
||
} | ||
else { | ||
freeCores(task.executorId) -= scheduler.CPUS_PER_TASK | ||
executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) | ||
} | ||
} | ||
} | ||
|
||
|
43 changes: 43 additions & 0 deletions
43
core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
/* | ||
* 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.scheduler | ||
|
||
import org.apache.spark.{LocalSparkContext, SparkConf, SparkException, SparkContext} | ||
import org.apache.spark.util.{SerializableBuffer, AkkaUtils} | ||
|
||
import org.scalatest.FunSuite | ||
|
||
class CoarseGrainedSchedulerBackendSuite extends FunSuite with LocalSparkContext { | ||
|
||
test("serialized task larger than akka frame size") { | ||
val conf = new SparkConf | ||
conf.set("spark.akka.frameSize","1") | ||
conf.set("spark.default.parallelism","1") | ||
sc = new SparkContext("local-cluster[2 , 1 , 512]", "test", conf) | ||
val frameSize = AkkaUtils.maxFrameSizeBytes(sc.conf) | ||
val buffer = new SerializableBuffer(java.nio.ByteBuffer.allocate(2 * frameSize)) | ||
val larger = sc.parallelize(Seq(buffer)) | ||
val thrown = intercept[SparkException] { | ||
larger.collect() | ||
} | ||
assert(thrown.getMessage.contains("Consider using broadcast variables for large values")) | ||
val smaller = sc.parallelize(1 to 4).collect() | ||
assert(smaller.size === 4) | ||
} | ||
|
||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
We try serializing a task here: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L767 to make sure everything is serializable. Can we do this check there as well -- or does that not work because of task size variability within a stage?