[
https://issues.apache.org/jira/browse/SPARK-4779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14290298#comment-14290298
]
Sven Krasser commented on SPARK-4779:
-------------------------------------
Here's a potentially related issue occurring on 1.2.0 on ElasticMapReduce/YARN
(however, this occurred when persisting to disk as opposed to a shuffle).
{noformat}
java.io.FileNotFoundException:
/mnt/var/lib/hadoop/tmp/nm-local-dir/usercache/hadoop/appcache/application_1421692415636_0059/spark-local-20150123232154-4cbf/3e/rdd_3_15108
(No such file or directory)
{noformat}
Full error:
{noformat}
2015-01-24 00:46:02,956 WARN [stdout writer for python] storage.MemoryStore
(Logging.scala:logWarning(71)) - Failed to reserve initial memory threshold of
1024.0 KB for computing block rdd_3_15108 in memory.
2015-01-24 00:46:02,957 WARN [stdout writer for python] storage.MemoryStore
(Logging.scala:logWarning(71)) - Not enough space to cache rdd_3_15108 in
memory! (computed 504.0 B so far)
2015-01-24 00:46:02,957 INFO [stdout writer for python] storage.MemoryStore
(Logging.scala:logInfo(59)) - Memory use = 2.2 GB (blocks) + 5.8 MB (scratch
space shared across 1 thread(s)) = 2.2 GB. Storage limit = 2.2 GB.
2015-01-24 00:46:02,957 WARN [stdout writer for python] spark.CacheManager
(Logging.scala:logWarning(71)) - Persisting partition rdd_3_15108 to disk
instead.
2015-01-24 00:46:02,958 WARN [stdout writer for python] storage.BlockManager
(Logging.scala:logWarning(71)) - Putting block rdd_3_15108 failed
2015-01-24 00:46:02,962 ERROR [stdout writer for python] python.PythonRDD
(Logging.scala:logError(96)) - Python worker exited unexpectedly (crashed)
org.apache.spark.api.python.PythonException: Traceback (most recent call last):
File
"/mnt1/var/lib/hadoop/tmp/nm-local-dir/usercache/hadoop/filecache/146/spark-assembly-1.2.0-hadoop2.4.0.jar/pyspark/worker.py",
line 90, in main
command = pickleSer._read_with_length(infile)
File
"/mnt1/var/lib/hadoop/tmp/nm-local-dir/usercache/hadoop/filecache/146/spark-assembly-1.2.0-hadoop2.4.0.jar/pyspark/serializers.py",
line 145, in _read_with_length
length = read_int(stream)
File
"/mnt1/var/lib/hadoop/tmp/nm-local-dir/usercache/hadoop/filecache/146/spark-assembly-1.2.0-hadoop2.4.0.jar/pyspark/serializers.py",
line 511, in read_int
raise EOFError
EOFError
at
org.apache.spark.api.python.PythonRDD$$anon$1.read(PythonRDD.scala:137)
at
org.apache.spark.api.python.PythonRDD$$anon$1.<init>(PythonRDD.scala:174)
at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:96)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:230)
at
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:93)
at
org.apache.spark.rdd.CoalescedRDD$$anonfun$compute$1.apply(CoalescedRDD.scala:92)
at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:350)
at scala.collection.Iterator$class.foreach(Iterator.scala:727)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
at
org.apache.spark.api.python.PythonRDD$.writeIteratorToStream(PythonRDD.scala:383)
at
org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply$mcV$sp(PythonRDD.scala:242)
at
org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:204)
at
org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:204)
at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1460)
at
org.apache.spark.api.python.PythonRDD$WriterThread.run(PythonRDD.scala:203)
Caused by: java.io.FileNotFoundException:
/mnt/var/lib/hadoop/tmp/nm-local-dir/usercache/hadoop/appcache/application_1421692415636_0059/spark-local-20150123232154-4cbf/3e/rdd_3_15108
(No such file or directory)
at java.io.FileOutputStream.open(Native Method)
at java.io.FileOutputStream.<init>(FileOutputStream.java:221)
at java.io.FileOutputStream.<init>(FileOutputStream.java:171)
at org.apache.spark.storage.DiskStore.putIterator(DiskStore.scala:75)
at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:787)
at
org.apache.spark.storage.BlockManager.putIterator(BlockManager.scala:638)
at
org.apache.spark.CacheManager.putInBlockManager(CacheManager.scala:145)
at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:70)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
... 5 more
2015-01-24 00:46:02,966 ERROR [stdout writer for python] python.PythonRDD
(Logging.scala:logError(96)) - This may have been caused by a prior exception:
java.io.FileNotFoundException:
/mnt/var/lib/hadoop/tmp/nm-local-dir/usercache/hadoop/appcache/application_1421692415636_0059/spark-local-20150123232154-4cbf/3e/rdd_3_15108
(No such file or directory)
at java.io.FileOutputStream.open(Native Method)
at java.io.FileOutputStream.<init>(FileOutputStream.java:221)
at java.io.FileOutputStream.<init>(FileOutputStream.java:171)
at org.apache.spark.storage.DiskStore.putIterator(DiskStore.scala:75)
at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:787)
at
org.apache.spark.storage.BlockManager.putIterator(BlockManager.scala:638)
at
org.apache.spark.CacheManager.putInBlockManager(CacheManager.scala:145)
at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:70)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:228)
at
org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply$mcV$sp(PythonRDD.scala:242)
at
org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:204)
at
org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:204)
at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1460)
at
org.apache.spark.api.python.PythonRDD$WriterThread.run(PythonRDD.scala:203)
{noformat}
> PySpark Shuffle Fails Looking for Files that Don't Exist when low on Memory
> ---------------------------------------------------------------------------
>
> Key: SPARK-4779
> URL: https://issues.apache.org/jira/browse/SPARK-4779
> Project: Spark
> Issue Type: Bug
> Components: PySpark, Shuffle
> Affects Versions: 1.1.0
> Environment: ec2 launched cluster with scripts
> 6 Nodes
> c3.2xlarge
> Reporter: Brad Willard
>
> When Spark is tight on memory it starts saying files don't exist during
> shuffle causing tasks to fail and be rebuilt destroying performance.
> The same code works flawlessly with smaller datasets with less memory
> pressure I assume.
> {code}
> 14/12/06 18:39:37 WARN scheduler.TaskSetManager: Lost task 292.0 in stage 3.0
> (TID 1099, ip-10-13-192-209.ec2.internal):
> org.apache.spark.api.python.PythonException: Traceback (most recent call
> last):
> File "/root/spark/python/pyspark/worker.py", line 79, in main
> serializer.dump_stream(func(split_index, iterator), outfile)
> File "/root/spark/python/pyspark/serializers.py", line 196, in dump_stream
> self.serializer.dump_stream(self._batched(iterator), stream)
> File "/root/spark/python/pyspark/serializers.py", line 127, in dump_stream
> for obj in iterator:
> File "/root/spark/python/pyspark/serializers.py", line 185, in _batched
> for item in iterator:
> File "/root/spark/python/pyspark/shuffle.py", line 370, in _external_items
> self.mergeCombiners(self.serializer.load_stream(open(p)),
> IOError: [Errno 2] No such file or directory:
> '/mnt/spark/spark-local-20141206182702-8748/python/16070/66618000/1/18'
>
> org.apache.spark.api.python.PythonRDD$$anon$1.read(PythonRDD.scala:124)
> org.apache.spark.api.python.PythonRDD$$anon$1.next(PythonRDD.scala:91)
> org.apache.spark.api.python.PythonRDD$$anon$1.next(PythonRDD.scala:87)
>
> org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43)
> scala.collection.Iterator$$anon$12.next(Iterator.scala:357)
>
> org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43)
> scala.collection.Iterator$$anon$12.next(Iterator.scala:357)
> scala.collection.Iterator$class.foreach(Iterator.scala:727)
> scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>
> org.apache.spark.api.python.PythonRDD$.writeIteratorToStream(PythonRDD.scala:335)
>
> org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply$mcV$sp(PythonRDD.scala:209)
>
> org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:184)
>
> org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:184)
> org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1311)
>
> org.apache.spark.api.python.PythonRDD$WriterThread.run(PythonRDD.scala:183)
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]