You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Cristian Opris (JIRA)" <ji...@apache.org> on 2014/11/12 16:21:33 UTC

[jira] [Comment Edited] (SPARK-3633) Fetches failure observed after SPARK-2711

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

Cristian Opris edited comment on SPARK-3633 at 11/12/14 3:20 PM:
-----------------------------------------------------------------

FWIW I get this as well, with a very straightforward job and setup.

Spark 1.1.0, executors configured to 2GB, storage.fraction=0.2, shuffle.spill=true

50GB dataset on ext4, spread over 7000 files, hence the coalescing below

The jobs is only doing: input.coalesce(72, false).groupBy(key).count

The groupBy is successful then I get the dreaded fetch error on count stage (oddly enough), but it seems to me that's when it does the actual shuffling for groupBy ?

EDIT: This might be due to Full GC on the executors during the shuffle block transfer phase. What's interesting is that it doesn't go OOM and the same amount is collected every time. (Old gen is 1.5 GB)

2014-11-12T07:17:06.899-0800: 477.697: [Full GC [PSYoungGen: 248320K->0K(466432K)] [ParOldGen: 1355469K->1301675K(1398272K)] 1603789K->1301675K(1864704K) [PSPermGen: 39031K->39031K(39424K)], 0.6565240 secs] [Times: user=3.35 sys=0.00, real=0.66 secs] 
2014-11-12T07:17:07.751-0800: 478.549: [Full GC [PSYoungGen: 248320K->0K(466432K)] [ParOldGen: 1301681K->1268312K(1398272K)] 1550001K->1268312K(1864704K) [PSPermGen: 39031K->39031K(39424K)], 0.5821160 secs] [Times: user=3.16 sys=0.00, real=0.58 secs] 
2014-11-12T07:17:08.495-0800: 479.294: [Full GC [PSYoungGen: 248320K->0K(466432K)] [ParOldGen: 1268314K->1300497K(1398272K)] 1516634K->1300497K(1864704K) [PSPermGen: 39031K->39031K(39424K)], 0.6400670 secs] [Times: user=4.07 sys=0.01, real=0.64 secs]



was (Author: onetoinfinity@yahoo.com):
FWIW I get this as well, with a very straightforward job and setup.

Spark 1.1.0, executors configured to 2GB, storage.fraction=0.2, shuffle.spill=true

50GB dataset on ext4, spread over 7000 files, hence the coalescing below

The jobs is only doing: input.coalesce(72, false).groupBy(key).count

The groupBy is successful then I get the dreaded fetch error on count stage (oddly enough), but it seems to me that's when it does the actual shuffling for groupBy ?



> 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: Critical
>
> 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-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org