[
https://issues.apache.org/jira/browse/SPARK-4759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14238289#comment-14238289
]
Andrew Or commented on SPARK-4759:
----------------------------------
I have a smaller reproduction for branch-1.1. It seems that we need to run the
two jobs in two different SparkContexts in tandem to reproduce it here:
1. Run bin/spark-shell. The master doesn't matter here.
2. Copy and paste the following into the REPL
{code}
def setup(): SparkContext = {
val conf = new SparkConf
conf.setMaster("local[8]")
conf.setAppName("test")
new SparkContext(conf)
}
def runMyJob(sc: SparkContext): Unit = {
val rdd = sc.parallelize(1 to 100).repartition(5).cache()
rdd.count()
val rdd2 = sc.parallelize(1 to 100).repartition(12)
rdd.union(rdd2).count()
}
def test(): Unit = {
var sc = setup()
runMyJob(sc)
sc.stop()
println("\n========== FINISHED FIRST JOB ==========\n")
sc = setup()
runMyJob(sc) // This will get stuck at task 5/17 and never finish
sc.stop()
println("\n========== FINISHED SECOND JOB ==========\n")
}
{code}
3. Call test().
Alternatively you can run the same code as an application in the source code
that I have attached. The benefit of not reproducing this through the
spark-shell is that there we don't start have more than one SparkContexts
running at any given time. Although I don't believe this has anything to do
with the root cause, it would be good to limit the scope of possible things
that could go wrong.
> Deadlock in complex spark job in local mode
> -------------------------------------------
>
> Key: SPARK-4759
> URL: https://issues.apache.org/jira/browse/SPARK-4759
> Project: Spark
> Issue Type: Bug
> Components: Spark Core
> Affects Versions: 1.1.1, 1.2.0, 1.3.0
> Environment: Java version "1.7.0_51"
> Java(TM) SE Runtime Environment (build 1.7.0_51-b13)
> Java HotSpot(TM) 64-Bit Server VM (build 24.51-b03, mixed mode)
> Mac OSX 10.10.1
> Using local spark context
> Reporter: Davis Shepherd
> Assignee: Andrew Or
> Priority: Critical
> Attachments: SparkBugReplicator.scala, SparkBugReplicatorSmaller.scala
>
>
> The attached test class runs two identical jobs that perform some iterative
> computation on an RDD[(Int, Int)]. This computation involves
> # taking new data merging it with the previous result
> # caching and checkpointing the new result
> # rinse and repeat
> The first time the job is run, it runs successfully, and the spark context is
> shut down. The second time the job is run with a new spark context in the
> same process, the job hangs indefinitely, only having scheduled a subset of
> the necessary tasks for the final stage.
> Ive been able to produce a test case that reproduces the issue, and I've
> added some comments where some knockout experimentation has left some
> breadcrumbs as to where the issue might be.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]