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

Stephen Haberman commented on SPARK-3633:
-----------------------------------------

Hi Josh,

Yes, it was GC issues; however, we're still kind of tracking down. Turns out 
the job that is failing is a real PITA, and is a month-to-date report, and 
started failing even on 0.9.2 at the end of last month.

We were able to get it to run on 0.9.2 with 8 i2.2xlarge machines (with 1.6gb 
SSDs) because we have a stage with ~1700 tasks, where the average shuffle write 
is ~400mb, but two of the tasks have huge schew, and shuffle writes of 15gb and 
125gb (!).

Since we got the rerun to work on the i2.2xlarges on 0.9.2, we have not been 
able to try the same data/job on 1.1.1/1.2, but it's on my list of things to do.

(Obviously we're going to want to solve this underlying schew issue as well, 
but, in theory :-), it should just lead to really slow jobs, vs. OOMEs/etc.)

> Fetches failure observed after SPARK-2711
> -----------------------------------------
>
>                 Key: SPARK-3633
>                 URL: https://issues.apache.org/jira/browse/SPARK-3633
>             Project: Spark
>          Issue Type: Bug
>          Components: Block Manager
>    Affects Versions: 1.1.0
>            Reporter: Nishkam Ravi
>            Priority: Blocker
>             Fix For: 1.1.1, 1.2.0
>
>
> Running a variant of PageRank on a 6-node cluster with a 30Gb input dataset. 
> Recently upgraded to Spark 1.1. The workload fails with the following error 
> message(s):
> {code}
> 14/09/19 12:10:38 WARN TaskSetManager: Lost task 51.0 in stage 2.1 (TID 552, 
> c1705.halxg.cloudera.com): FetchFailed(BlockManagerId(1, 
> c1706.halxg.cloudera.com, 49612, 0), shuffleId=3, mapId=75, reduceId=120)
> 14/09/19 12:10:38 INFO DAGScheduler: Resubmitting failed stages
> {code}
> In order to identify the problem, I carried out change set analysis. As I go 
> back in time, the error message changes to:
> {code}
> 14/09/21 12:56:54 WARN TaskSetManager: Lost task 35.0 in stage 3.0 (TID 519, 
> c1706.halxg.cloudera.com): java.io.FileNotFoundException: 
> /var/lib/jenkins/workspace/tmp/spark-local-20140921123257-68ee/1c/temp_3a1ade13-b48a-437a-a466-673995304034
>  (Too many open files)
>         java.io.FileOutputStream.open(Native Method)
>         java.io.FileOutputStream.<init>(FileOutputStream.java:221)
>         
> org.apache.spark.storage.DiskBlockObjectWriter.open(BlockObjectWriter.scala:117)
>         
> org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:185)
>         
> org.apache.spark.util.collection.ExternalAppendOnlyMap.spill(ExternalAppendOnlyMap.scala:197)
>         
> org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:145)
>         org.apache.spark.Aggregator.combineValuesByKey(Aggregator.scala:58)
>         
> org.apache.spark.shuffle.hash.HashShuffleWriter.write(HashShuffleWriter.scala:51)
>         
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:68)
>         
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>         org.apache.spark.scheduler.Task.run(Task.scala:54)
>         org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199)
>         
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         java.lang.Thread.run(Thread.java:745)
> {code}
> All the way until Aug 4th. Turns out the problem changeset is 4fde28c. 



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

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

Reply via email to