too many open files - shuffle

2,473 views
Skip to first unread message

John Carnahan

unread,
Oct 23, 2013, 8:40:03 PM10/23/13
to spark...@googlegroups.com
My jobs are hitting "too many open files" exceptions. I am using 0.8.0 and the cause seems to be a very large number of shuffle files (in /tmp/spark-local*) which remain open after successive iterations. During each successive run the number of files there grows to my ulimit - at which point my job crashes. I don't remember this problem in 0.7. This seems to be related to: https://groups.google.com/d/msg/spark-users/EkbeCGFDaAQ/mL6t9lXsPYcJ

 Is there are workaround for this problem?



Matei Zaharia

unread,
Oct 23, 2013, 9:24:12 PM10/23/13
to spark...@googlegroups.com
Hi John,

What's your ulimit? You can safely increase the ulimit to 100000 or even more. Just add ulimit -n 100000 in conf/spark-env.sh.

As far as I know, the files shouldn't stay open across iterations -- try doing lsof on the process to see if they are. But the files for just one iteration may be too many. In Spark 0.8.1, we have a patch that decreases the number of files.

Matei

On Oct 23, 2013, at 5:40 PM, John Carnahan <jmc...@gmail.com> wrote:

My jobs are hitting "too many open files" exceptions. I am using 0.8.0 and the cause seems to be a very large number of shuffle files (in /tmp/spark-local*) which remain open after successive iterations. During each successive run the number of files there grows to my ulimit - at which point my job crashes. I don't remember this problem in 0.7. This seems to be related to: https://groups.google.com/d/msg/spark-users/EkbeCGFDaAQ/mL6t9lXsPYcJ

 Is there are workaround for this problem?




--
You received this message because you are subscribed to the Google Groups "Spark Users" group.
To unsubscribe from this group and stop receiving emails from it, send an email to spark-users...@googlegroups.com.
For more options, visit https://groups.google.com/groups/opt_out.

John Carnahan

unread,
Oct 23, 2013, 9:39:44 PM10/23/13
to spark...@googlegroups.com, spark...@googlegroups.com
I am using a 64k ulimit. Each iteration adds about 20k new open shuffle* files on each machine and keeps growing. Iterations are either not cleaning things up or cleaning up too slowly for the size of my iterations. Is the clean up of these files asynchronous? Should i be using netty shuffle or no? Is there something else I could be doing wrong? 
You received this message because you are subscribed to a topic in the Google Groups "Spark Users" group.
To unsubscribe from this topic, visit https://groups.google.com/d/topic/spark-users/ofaLPxlPi_8/unsubscribe.
To unsubscribe from this group and all its topics, send an email to spark-users...@googlegroups.com.

Matei Zaharia

unread,
Oct 23, 2013, 9:56:27 PM10/23/13
to spark...@googlegroups.com
I see, I'm not sure what's keeping them open then, but one possibility is that the garbage collector might need to kick in to free some memory-mapped buffer objects, so that might not happen frequently enough. I'd recommend trying with a higher ulimit. Alternatively, the master branch of the Spark repo now has the shuffle file consolidation patch in it, which will greatly reduce the number of files.

The files themselves are actually not deleted while the job is running by the way, but that shouldn't cause file handles to be open. It's also possible that there's a file handle leak.

Matei

Reynold Xin

unread,
Oct 23, 2013, 9:57:43 PM10/23/13
to spark...@googlegroups.com
The files are not deleted, but they are closed.

I just tested on the Spark 0.8.0 release by doing: 

sc.parallelize(1 to 1000, 2).map(x => (x, x)).reduceByKey(_+_, 1000).count


rxin @ dhcpx-193-92 : ~/Downloads/spark-0.8.0-incubating-bin-hadoop1 

> lsof -p 2099 | wc -l

      88        // before the job started

rxin @ dhcpx-193-92 : ~/Downloads/spark-0.8.0-incubating-bin-hadoop1 

> lsof -p 2099 | wc -l

     305        // when the job was running

rxin @ dhcpx-193-92 : ~/Downloads/spark-0.8.0-incubating-bin-hadoop1 

> lsof -p 2099 | wc -l

      88          // after the job finished


The files were closed properly it looks like.

Matei Zaharia

unread,
Oct 23, 2013, 10:08:19 PM10/23/13
to spark...@googlegroups.com
BTW there's also a small chance that this is JVM-specific since we use a sun.* API to close memory-mapped files, so it would be great to learn more about your environment and app.

Matei

Stephen Haberman

unread,
Oct 24, 2013, 2:43:50 AM10/24/13
to spark...@googlegroups.com, matei....@gmail.com

Well, unfortunately I have to chime in here...

We've been running 0.7.x for awhile, but tonight I tried 0.8 and got
this too many files error. Still poking at it...

- Stephen

Stephen Haberman

unread,
Oct 24, 2013, 2:53:02 AM10/24/13
to spark...@googlegroups.com, matei....@gmail.com

> Well, unfortunately I have to chime in here...
>
> We've been running 0.7.x for awhile, but tonight I tried 0.8 and got
> this too many files error. Still poking at it...

Gah! Sorry, I hit send in the wrong email.

I meant to type "Still poking at it"+send in an email to my co-worker
who's working with me on this, not the email to the list where I was
gathering my thoughts. :-)

Nevermind, for now I'm still playing with ulimit/etc.

- Stephen

Message has been deleted

John Carnahan

unread,
Oct 24, 2013, 12:23:59 PM10/24/13
to spark...@googlegroups.com
Reynold


> lsof -p 2099 | wc -l

      88          // after the job finished

The files were closed properly it looks like.


Right. What you describe works fine. When the job is complete the files are closed. My problem is that the files used in any given iteration are not closed. I have a job that traverses a large RDD once in each iteration. Nothing is persisted except the single large RDD. After a few iterations my job crashes because of too many open files. This seems like it would be a problem for everyone if iterating over an RDD progressively increases a load of open file pointers.

Message has been deleted

John Carnahan

unread,
Oct 24, 2013, 12:36:56 PM10/24/13
to spark...@googlegroups.com
Matei


it would be great to learn more about your environment and app.

Pretty standard centos distro

> 2.6.18-308.0.0.0.1.el5xen #1 SMP Sat Feb 25 16:26:29 EST 2012 x86_64 x86_64 x86_64 GNU/Linux

My JVM is

> java version "1.6.0_34"
> Java(TM) SE Runtime Environment (build 1.6.0_34-b04)
> Java HotSpot(TM) 64-Bit Server VM (build 20.9-b04, mixed mode)

Reynold Xin

unread,
Oct 24, 2013, 12:43:27 PM10/24/13
to spark...@googlegroups.com
I tried this again and couldn't reproduce on my Macbook Air. 

sc.parallelize(1 to 1000, 2).map(x => (x, x)).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).count

using a local cluster (local-cluster[2,1,512]) so I have two workers and one master.

When this is running, the driver application process stays at 163 files opened, and each of the worker process stays at 96 files opened. 

If you run the above program in your cluster, do you see this problem? 



On Thu, Oct 24, 2013 at 9:36 AM, John Carnahan <jmc...@gmail.com> wrote:
Matei


it would be great to learn more about your environment and app.

Pretty standard centos distro

> 2.6.18-308.0.0.0.1.el5xen #1 SMP Sat Feb 25 16:26:29 EST 2012 x86_64 x86_64 x86_64 GNU/Linux

My JVM is

> java version "1.6.0_34"
> Java(TM) SE Runtime Environment (build 1.6.0_34-b04)
> Java HotSpot(TM) 64-Bit Server VM (build 20.9-b04, mixed mode)

John Carnahan

unread,
Oct 24, 2013, 12:57:06 PM10/24/13
to spark...@googlegroups.com
I am new to the spark code base but in looking at ShuffleMapTask.scala there is a finally clause to clean up the block writers after the shuffle is complete. In this clause - ShuffleBlockManager.releaseWriters() is called with buckets that were opened. However in ShuffleBlockManager the releaseWriters() method states

      override def releaseWriters(group: ShuffleWriterGroup) = {
        // Nothing really to release here.
      }

Could this be the source of the problem? It doesn't look like there is some other async process that closes the files unless I am missing a link somewhere.

Reynold Xin

unread,
Oct 24, 2013, 1:01:33 PM10/24/13
to spark...@googlegroups.com
Yea we fixed that for 0.8.1 but the problem exists for 0.8.0. However, the problem (of not closing the files) is only triggered if there is an error in the execution. 


Yann Luppo

unread,
Oct 24, 2013, 1:58:54 PM10/24/13
to spark...@googlegroups.com
Hi,

When checking the source on github (https://github.com/apache/incubator-spark) we get 0.9.0 and the apache site doesn't offer to download 0.8.1 yet. 
To fix this issue should we download the latest from git (0.9.0) ? Or is there another place to get 0.8.1?

Thanks.

Reynold Xin

unread,
Oct 24, 2013, 2:05:59 PM10/24/13
to spark...@googlegroups.com
0.8.1 is not released yet. And I don't think that is the cause of this anyway.


John Carnahan

unread,
Oct 25, 2013, 2:44:55 PM10/25/13
to spark...@googlegroups.com
Here is the latest. In an isolated instance of spark (i.e. local-cluster[2,1,512]) on the same machine I ran the what Reynold suggested

sc.parallelize(1 to 1000, 2).map(x => (x, x)).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).reduceByKey(_+_, 1000).map(x => x).count

The result
1. lots of shuffle files were created
2. the number of open files according to lsof remained stable across iterations

That is expected. I then reran my application on the cluster and achieved similar results - lots of shuffle files and a stable number of open files across iterations according to lsof. However at about the same time the number of shuffle files exceeded my ulimit I began to get 'too many open files' exceptions until the app died.

This suggests that, in fact, there is something else going on than open files as observed by lsof. I will investigate further why I might be getting the 'too many open files' error

Reynold Xin

unread,
Oct 25, 2013, 2:48:55 PM10/25/13
to spark...@googlegroups.com
Thanks, John. If there is indeed a problem, we should absolutely fix this for 0.8.1. It would be great if you could look into it. If you need help, please let us know.


Matei Zaharia

unread,
Oct 25, 2013, 3:33:03 PM10/25/13
to spark...@googlegroups.com
Thanks for trying this out. If you can post more details about your application (e.g. what the sequence of operations is), that would help us too.

Matei

Reply all
Reply to author
Forward
0 new messages