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

Doug Cutting commented on HADOOP-1179:
--------------------------------------

Note that running out of file handles can cause OutOfMemoryExceptions, even 
when there's lots of memory left.  So this could just be a file-handle leak.  
It'd be great to log the heap size somehow (e.g. simply by turning on 
verbosegc, which has little performance impact) so we had an idea of whether 
this is really a memory issue or a file handle issue.

> task Tracker should be restarted if its jetty http server cannot serve 
> get-map-output files
> -------------------------------------------------------------------------------------------
>
>                 Key: HADOOP-1179
>                 URL: https://issues.apache.org/jira/browse/HADOOP-1179
>             Project: Hadoop
>          Issue Type: Bug
>            Reporter: Runping Qi
>         Assigned To: Devaraj Das
>         Attachments: 1179.patch
>
>
> Due to some errors (mem leak?), the jetty http server throws outOfMemory 
> exception when serving get-map-output requests:
> 2007-03-28 20:42:39,608 WARN org.mortbay.jetty.servlet.ServletHandler: Error 
> for
>  /mapOutput?map=task_0334_m_013127_0&reduce=591
> 2007-03-28 20:46:42,788 WARN org.mortbay.jetty.servlet.ServletHandler: Error 
> for
>  /mapOutput?map=task_0334_m_013127_0&reduce=591
> 2007-03-28 20:49:38,064 WARN org.mortbay.jetty.servlet.ServletHandler: Error 
> for
> java.lang.OutOfMemoryError
>         at java.io.FileInputStream.readBytes(Native Method)
>         at java.io.FileInputStream.read(FileInputStream.java:199)
>         at 
> org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileInputStream.read(R
> awLocalFileSystem.java:119)
>         at 
> org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInput
> Stream.java:41)
>         at java.io.BufferedInputStream.read1(BufferedInputStream.java:256)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:317)
>         at java.io.DataInputStream.read(DataInputStream.java:132)
>         at 
> org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(Che
> cksumFileSystem.java:182)
>         at 
> org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumF
> ileSystem.java:167)
>         at 
> org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInput
> Stream.java:41)
>         at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>         at java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:317)
>         at java.io.DataInputStream.readFully(DataInputStream.java:178)
>         at java.io.DataInputStream.readLong(DataInputStream.java:399)
>         at 
> org.apache.hadoop.mapred.TaskTracker$MapOutputServlet.doGet(TaskTrack
> er.java:1643)
>         at javax.servlet.http.HttpServlet.service(HttpServlet.java:689)
>         at javax.servlet.http.HttpServlet.service(HttpServlet.java:802)
>         at 
> org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder.java:427
> )
>         at 
> org.mortbay.jetty.servlet.WebApplicationHandler.dispatch(WebApplicati
> onHandler.java:475)
>         at 
> org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandler.java:5
> 67)
>         at org.mortbay.http.HttpContext.handle(HttpContext.java:1565)
>         at 
> org.mortbay.jetty.servlet.WebApplicationContext.handle(WebApplication
> Context.java:635)
>         at org.mortbay.http.HttpContext.handle(HttpContext.java:1517)
>         at org.mortbay.http.HttpServer.service(HttpServer.java:954)
>         at org.mortbay.http.HttpConnection.service(HttpConnection.java:814)
>         at org.mortbay.http.HttpConnection.handleNext(HttpConnection.java:981)
>         at org.mortbay.http.HttpConnection.handle(HttpConnection.java:831)
>         at 
> org.mortbay.http.SocketListener.handleConnection(SocketListener.java:
> 244)
>         at org.mortbay.util.ThreadedServer.handle(ThreadedServer.java:357)
>         at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:534)
> In this case, the task tracker cannot send out the map outut files on that 
> machine, rendering it useless.
> Moreover, all the reduces depending on those map output files are just stuck 
> there.
> If the task tracker reports fail to the job tracker, the map/reduce job can 
> recover.
> If the task tracker restarted, it can continue to join the cluster as a new 
> mamber.

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

Reply via email to