Return-Path: Delivered-To: apmail-lucene-hadoop-dev-archive@locus.apache.org Received: (qmail 96437 invoked from network); 29 Mar 2007 14:51:56 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 29 Mar 2007 14:51:56 -0000 Received: (qmail 18547 invoked by uid 500); 29 Mar 2007 14:51:55 -0000 Delivered-To: apmail-lucene-hadoop-dev-archive@lucene.apache.org Received: (qmail 18527 invoked by uid 500); 29 Mar 2007 14:51:55 -0000 Mailing-List: contact hadoop-dev-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hadoop-dev@lucene.apache.org Delivered-To: mailing list hadoop-dev@lucene.apache.org Received: (qmail 18498 invoked by uid 99); 29 Mar 2007 14:51:54 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 29 Mar 2007 07:51:54 -0700 X-ASF-Spam-Status: No, hits=-100.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO brutus.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 29 Mar 2007 07:51:45 -0700 Received: from brutus (localhost [127.0.0.1]) by brutus.apache.org (Postfix) with ESMTP id 4EA95714065 for ; Thu, 29 Mar 2007 07:51:25 -0700 (PDT) Message-ID: <23668830.1175179885312.JavaMail.jira@brutus> Date: Thu, 29 Mar 2007 07:51:25 -0700 (PDT) From: "Runping Qi (JIRA)" To: hadoop-dev@lucene.apache.org Subject: [jira] Commented: (HADOOP-1179) task Tracker should be restarted if its jetty http server cannot serve get-map-output files In-Reply-To: <32055443.1175119825209.JavaMail.jira@brutus> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/HADOOP-1179?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12485201 ] Runping Qi commented on HADOOP-1179: ------------------------------------ This issue is related to HADOOP-1158 but not a dup. Two things need to be investigated/fixed: why out of memory in Jetty (or in TT). Early comments and the attached patch may address that partially. However, I am not sure that accounts all. The second thing is proper handling of outOfMemory. Normally, when this happens, it may be desirable to kill the TT and hope to be restarted. > 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.