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

Josh Rosen commented on SPARK-9499:
-----------------------------------

Looking at the {{lsof}} dump attached to the original report at SPARK-8850, it 
looks like most of these open files are shuffle data:

{code}
java    29136 hvanhovell 1959r   REG              252,1   2810696 20717588 
/tmp/blockmgr-b6c982a0-1230-498f-b997-e7d24027eb43/30/shuffle_19_39_0.data
java    29136 hvanhovell 1960r   REG              252,1   2847096 20718508 
/tmp/blockmgr-b6c982a0-1230-498f-b997-e7d24027eb43/19/shuffle_19_1_0.data
[...]
{code}


 I'm going to see if I can reproduce this issue locally; I'm not yet convinced 
that it's necessarily Tungsten related, but let's see.

> Possible file handle leak in spilling/sort code
> -----------------------------------------------
>
>                 Key: SPARK-9499
>                 URL: https://issues.apache.org/jira/browse/SPARK-9499
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>            Reporter: Reynold Xin
>            Assignee: Josh Rosen
>            Priority: Blocker
>
> As reported by [~hvanhovell]. See SPARK-8850.
> Hi,
> I am getting a Too many open files error since the unsafe mode is on. The 
> same thing popped up when playing with unsafe before. The error is below:
> {noformat}
> 15/07/30 23:37:29 WARN TaskSetManager: Lost task 2.0 in stage 33.0 (TID 2423, 
> localhost): java.io.FileNotFoundException: 
> /tmp/blockmgr-b3d3e14a-f313-4075-8082-7d97f012e35a/14/temp_shuffle_1cab42fa-dcb1-4114-ae53-1674446f9dac
>  (Too many open files)
>       at java.io.FileOutputStream.open0(Native Method)
>       at java.io.FileOutputStream.open(FileOutputStream.java:270)
>       at java.io.FileOutputStream.<init>(FileOutputStream.java:213)
>       at 
> org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:88)
>       at 
> org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.insertAll(BypassMergeSortShuffleWriter.java:111)
>       at 
> org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:71)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:86)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> I am currently working on local mode (which is probably the cause of the 
> problem) using the following command line:
> {noformat}
> $SPARK_HOME/bin/spark-shell --master local[*] --driver-memory 14G 
> --driver-library-path $HADOOP_NATIVE_LIB
> {noformat}
> The maximum number of files I can open are 1024 (ulimit -n). I have tried to 
> run the same code with an increased limit, but this didn't work out.
> Dump of all open files after a Too Many Files Open error.
> The command used to make the dump:
> {code}
> lsof -c java > open
> {code}
> The job starts crashing after as soon as I start sorting 10000000 rows for 
> the 9th time (doing benchmarking). I guess files are left open after every 
> benchmark? Is there a way to trigger the closing of files?



--
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