spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sean Owen (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-17777) Spark Scheduler Hangs Indefinitely
Date Wed, 05 Oct 2016 14:12:20 GMT

    [ https://issues.apache.org/jira/browse/SPARK-17777?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15548821#comment-15548821
] 

Sean Owen commented on SPARK-17777:
-----------------------------------

You could do a kill -QUIT on the driver to dump thread stacks see what's waiting where, and
that would probably point to the actual issue. I suspect we'd find there's some deadlock in
this case, because you're trying to schedule and execute jobs within a method that's called
within the scheduler itself, in places where this is assumed to not be happening. I don't
know if that's the issue or if so, whether it's a deep assumption or not.

Why isn't it documented? I suppose there are a hundred things you can't do here. My strong
assumption is that this would never be intended to work, but then again I stare at the code
daily. But that depends on an assumption about why this isn't working.

If you just mean to list resources in parallel, just do so in parallel from the driver, without
trying to send things across the cluster. That strikes me as the more natural way to do this.
That can be lazy and in getPartitions.

> Spark Scheduler Hangs Indefinitely
> ----------------------------------
>
>                 Key: SPARK-17777
>                 URL: https://issues.apache.org/jira/browse/SPARK-17777
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 1.6.0
>         Environment: AWS EMR 4.3, can also be reproduced locally
>            Reporter: Ameen Tayyebi
>         Attachments: repro.scala
>
>
> We've identified a problem with Spark scheduling. The issue manifests itself when an
RDD calls SparkContext.parallelize within its getPartitions method. This seemingly "recursive"
call causes the problem. We have a repro case that can easily be run.
> Please advise on what the issue might be and how we can work around it in the mean time.
> I've attached repro.scala which can simply be pasted in spark-shell to reproduce the
problem.
> Why are we calling sc.parallelize in production within getPartitions? Well, we have an
RDD that is composed of several thousands of Parquet files. To compute the partitioning strategy
for this RDD, we create an RDD to read all file sizes from S3 in parallel, so that we can
quickly determine the proper partitions. We do this to avoid executing this serially from
the master node which can result in significant slowness in the execution. Pseudo-code:
> val splitInfo = sc.parallelize(filePaths).map(f => (f, s3.getObjectSummary)).collect()
> A similar logic is used in DataFrame by Spark itself:
> https://github.com/apache/spark/blob/branch-1.6/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala#L902

> Thanks,
> -Ameen



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org


Mime
View raw message