hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Devaraj Das (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HADOOP-4614) "Too many open files" error while processing a large gzip file
Date Wed, 12 Nov 2008 16:57:44 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-4614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12646951#action_12646951
] 

Devaraj Das commented on HADOOP-4614:
-------------------------------------

I took one look at the mergeParts code and didn't see any obvious issue with that. What's
your setting for io.sort.factor and io.sort.mb? Also, could you please run the same job again
with the exact (default?) setting for the Max FDs that resulted in this problem, and this
time could you do "lsof -p <pid-of-map-task-process>". Please post the results here.
Thanks!

> "Too many open files" error while processing a large gzip file
> --------------------------------------------------------------
>
>                 Key: HADOOP-4614
>                 URL: https://issues.apache.org/jira/browse/HADOOP-4614
>             Project: Hadoop Core
>          Issue Type: Bug
>          Components: mapred
>    Affects Versions: 0.18.2
>            Reporter: Abdul Qadeer
>             Fix For: 0.18.3
>
>         Attachments: openfds.txt
>
>
> I am running a simple word count program on a gzip compressed data of size 4 GB (Uncompressed
size is about 7 GB).  I have setup of 17 nodes in my Hadoop cluster.  After some time, I get
the following exception:
> java.io.FileNotFoundException: /usr/local/hadoop/hadoop-hadoop/mapred/local/taskTracker/jobcache/job_200811041109_0003/attempt_200811041109_0003_m_000000_0/output/spill4055.out.index
> (Too many open files)
>        at java.io.FileInputStream.open(Native Method)
>        at java.io.FileInputStream.(FileInputStream.java:137)
>        at org.apache.hadoop.fs.RawLocalFileSystem$TrackingFileInputStream.(RawLocalFileSystem.java:62)
>        at org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileInputStream.(RawLocalFileSystem.java:98)
>        at org.apache.hadoop.fs.RawLocalFileSystem.open(RawLocalFileSystem.java:168)
>        at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:359)
>        at org.apache.hadoop.mapred.IndexRecord.readIndexFile(IndexRecord.java:47)
>        at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.getIndexInformation(MapTask.java:1339)
>        at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.mergeParts(MapTask.java:1237)
>        at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:857)
>        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:333)
>        at org.apache.hadoop.mapred.Child.main(Child.java:155)
> From a user's perspective I know that Hadoop will use only one mapper for a gzipped file.
 The above exception suggests that probably Hadoop puts the intermediate data into many files.
 But the question is that "exactly how many open files are required in the worst case for
any data size and cluster size?"  Currently it looks as if Hadoop needs more number of open
files as the size of input or the cluster size (in terms of nodes, mapper, reducers) increases.
 This is not plausible as far as scalability is concerned.  A user needs to write some number
in the /etc/security/limits.conf file that how many open files are allowed by hadoop node.
 The question is what that "magical number" should be?
> So probably the best solution to this problem is to change Hadoop such a way that it
can work with some moderate number of allowed open files (e.g. 4 K) or any other number should
be suggested as an upper limit such that a user is sure that for any data size and cluster
size, hadoop will not run into this "too many open files" issue.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message