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-874) merge code is really slow
Date Sat, 13 Jan 2007 11:29:27 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12464437
] 

Devaraj Das commented on HADOOP-874:
------------------------------------

Adding the problem raised by Bryan to keep the discussion on this issue in one place.

> -----Original Message-----
> From: bpendleton@gmail.com [mailto:bpendleton@gmail.com] On Behalf Of
> Bryan A. P. Pendleton
> Sent: Tuesday, January 09, 2007 1:33 AM
> To: hadoop-dev@lucene.apache.org
> Subject: Problems with running out of file handles?
> 
> I've recently added a bunch of big new machines to my cluster.
> 
> However, it seems their defaults for number of files open per user is
> lower.
> Though I know I can fix this, I thought the "spilling" code was supposed
> to
> prevent running out of open files? Are spills not always closed after
> they've been written to, or, is there some other growth in the number of
> open output files?
> 
> The machines have 1024 files open per user by default, and have been
> running
> out of file descriptors (and erroring out the tasks run on them) at
> spill253.out. For instance:
> 
> java.io.FileNotFoundException:
> /state/partition1/hadoop/mapred/local/task_0001_m_000077_0/spill253.out
> (Too
> many open files)
>     at java.io.FileInputStream.open(Native Method)
>     at java.io.FileInputStream.(FileInputStream.java:106)
>     at org.apache.hadoop.fs.LocalFileSystem$LocalFSFileInputStream.(
> LocalFileSystem.java:92)
>     at org.apache.hadoop.fs.LocalFileSystem.openRaw(LocalFileSystem.java
> :143)
>     at org.apache.hadoop.fs.FSDataInputStream$Checker.(
> FSDataInputStream.java:52)
>     at org.apache.hadoop.fs.FSDataInputStream.(FSDataInputStream.java:279)
>     at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:262)
>     at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.mergeParts(
> MapTask.java:475)
>     at org.apache.hadoop.mapred.MapTask.run(MapTask.java:191)
>     at org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java
> :1347)
> 
> 
> Before anyone asks, yes, my map outputs have *huge* outputs (~1Tb/90 tasks
> ~= 10Gb each).
> 
> The problem appears to be the mergeParts job, which looks like it opens up
> *all* spill files when its merging. Wasn't this code supposed to reduce
> file-descriptor pressure? Seems like, in this case, if I had only 30
> reducers, I'd've been capped at 30 file descriptors per map reduce task.
> Now, it appears to need outputsize/spillsize file descriptors, which
> *could*
> be >> 30.
> 
> --
> Bryan A. P. Pendleton
> Ph: (877) geek-1-bp


> merge code is really slow
> -------------------------
>
>                 Key: HADOOP-874
>                 URL: https://issues.apache.org/jira/browse/HADOOP-874
>             Project: Hadoop
>          Issue Type: Bug
>          Components: io
>    Affects Versions: 0.10.0
>            Reporter: Owen O'Malley
>         Assigned To: Devaraj Das
>             Fix For: 0.11.0
>
>         Attachments: merge-no-seek.patch
>
>
> I had a case where the map output buffer size (io.sort.mb) was set too low and caused
a spill and merge. Fixing the configuration caused it to not spill until it was finished.
With the spill it took 9.5 minutes per a map. Without the spill it took 45 seconds. Therefore,
I assume it was taking ~9 minutes to do the 2 file merge. That is really slow. The input files
to the merge were two 25 mb sequence files (default codec (java), block compressed)

-- 
This message is automatically generated by JIRA.
-
If you think it was sent incorrectly contact one of the administrators: https://issues.apache.org/jira/secure/Administrators.jspa
-
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message