Return-Path: Delivered-To: apmail-hadoop-core-user-archive@www.apache.org Received: (qmail 71070 invoked from network); 8 Sep 2008 10:40:38 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 8 Sep 2008 10:40:38 -0000 Received: (qmail 38174 invoked by uid 500); 8 Sep 2008 10:40:30 -0000 Delivered-To: apmail-hadoop-core-user-archive@hadoop.apache.org Received: (qmail 38115 invoked by uid 500); 8 Sep 2008 10:40:29 -0000 Mailing-List: contact core-user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: core-user@hadoop.apache.org Delivered-To: mailing list core-user@hadoop.apache.org Received: (qmail 38104 invoked by uid 99); 8 Sep 2008 10:40:28 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 08 Sep 2008 03:40:28 -0700 X-ASF-Spam-Status: No, hits=1.2 required=10.0 tests=SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (athena.apache.org: local policy) Received: from [72.14.220.156] (HELO fg-out-1718.google.com) (72.14.220.156) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 08 Sep 2008 10:39:31 +0000 Received: by fg-out-1718.google.com with SMTP id l26so1348303fgb.35 for ; Mon, 08 Sep 2008 03:40:01 -0700 (PDT) Received: by 10.86.70.3 with SMTP id s3mr11175293fga.51.1220870401210; Mon, 08 Sep 2008 03:40:01 -0700 (PDT) Received: by 10.86.26.6 with HTTP; Mon, 8 Sep 2008 03:40:01 -0700 (PDT) Message-ID: Date: Mon, 8 Sep 2008 12:40:01 +0200 From: "Espen Amble Kolstad" To: core-user@hadoop.apache.org Subject: Re: Could not obtain block: blk_-2634319951074439134_1129 file=/user/root/crawl_debug/segments/20080825053518/content/part-00002/data In-Reply-To: <200809081124.52946.espen@trank.no> MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 7bit Content-Disposition: inline References: <4aa34eb70809070042g2df4c9c6r3f2ab137e16dbd53@mail.gmail.com> <200809081124.52946.espen@trank.no> X-Virus-Checked: Checked by ClamAV on apache.org There's a JIRA on this already: https://issues.apache.org/jira/browse/HADOOP-3831 Setting dfs.datanode.socket.write.timeout=0 in hadoop-site.xml seems to do the trick for now. Espen On Mon, Sep 8, 2008 at 11:24 AM, Espen Amble Kolstad wrote: > Hi, > > Thanks for the tip! > > I tried revision 692572 of the 0.18 branch, but I still get the same errors. > > On Sunday 07 September 2008 09:42:43 Dhruba Borthakur wrote: >> The DFS errors might have been caused by >> >> http://issues.apache.org/jira/browse/HADOOP-4040 >> >> thanks, >> dhruba >> >> On Sat, Sep 6, 2008 at 6:59 AM, Devaraj Das wrote: >> > These exceptions are apparently coming from the dfs side of things. Could >> > someone from the dfs side please look at these? >> > >> > On 9/5/08 3:04 PM, "Espen Amble Kolstad" wrote: >> >> Hi, >> >> >> >> Thanks! >> >> The patch applies without change to hadoop-0.18.0, and should be >> >> included in a 0.18.1. >> >> >> >> However, I'm still seeing: >> >> in hadoop.log: >> >> 2008-09-05 11:13:54,805 WARN dfs.DFSClient - Exception while reading >> >> from blk_3428404120239503595_2664 of >> >> /user/trank/segments/20080905102650/crawl_generate/part-00010 from >> >> somehost:50010: java.io.IOException: Premeture EOF from in >> >> putStream >> >> >> >> in datanode.log: >> >> 2008-09-05 11:15:09,554 WARN dfs.DataNode - >> >> DatanodeRegistration(somehost:50010, >> >> storageID=DS-751763840-somehost-50010-1219931304453, infoPort=50075, >> >> ipcPort=50020):Got exception while serving >> >> blk_-4682098638573619471_2662 to >> >> /somehost: >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting >> >> for channel to be ready for write. ch : >> >> java.nio.channels.SocketChannel[connected local=/somehost:50010 >> >> remote=/somehost:45244] >> >> >> >> These entries in datanode.log happens a few minutes apart repeatedly. >> >> I've reduced # map-tasks so load on this node is below 1.0 with 5GB of >> >> free memory (so it's not resource starvation). >> >> >> >> Espen >> >> >> >> On Thu, Sep 4, 2008 at 3:33 PM, Devaraj Das wrote: >> >>>> I started a profile of the reduce-task. I've attached the profiling >> >>>> output. It seems from the samples that ramManager.waitForDataToMerge() >> >>>> doesn't actually wait. >> >>>> Has anybody seen this behavior. >> >>> >> >>> This has been fixed in HADOOP-3940 >> >>> >> >>> On 9/4/08 6:36 PM, "Espen Amble Kolstad" wrote: >> >>>> I have the same problem on our cluster. >> >>>> >> >>>> It seems the reducer-tasks are using all cpu, long before there's >> >>>> anything to >> >>>> shuffle. >> >>>> >> >>>> I started a profile of the reduce-task. I've attached the profiling >> >>>> output. It seems from the samples that ramManager.waitForDataToMerge() >> >>>> doesn't actually wait. >> >>>> Has anybody seen this behavior. >> >>>> >> >>>> Espen >> >>>> >> >>>> On Thursday 28 August 2008 06:11:42 wangxu wrote: >> >>>>> Hi,all >> >>>>> I am using hadoop-0.18.0-core.jar and nutch-2008-08-18_04-01-55.jar, >> >>>>> and running hadoop on one namenode and 4 slaves. >> >>>>> attached is my hadoop-site.xml, and I didn't change the file >> >>>>> hadoop-default.xml >> >>>>> >> >>>>> when data in segments are large,this kind of errors occure: >> >>>>> >> >>>>> java.io.IOException: Could not obtain block: >> >>>>> blk_-2634319951074439134_1129 >> >>>>> file=/user/root/crawl_debug/segments/20080825053518/content/part-0000 >> >>>>>2/data at >> >>>>> org.apache.hadoop.dfs.DFSClient$DFSInputStream.chooseDataNode(DFSClie >> >>>>>nt.jav a:1462) at >> >>>>> org.apache.hadoop.dfs.DFSClient$DFSInputStream.blockSeekTo(DFSClient. >> >>>>>java:1 312) at >> >>>>> org.apache.hadoop.dfs.DFSClient$DFSInputStream.read(DFSClient.java:14 >> >>>>>17) at java.io.DataInputStream.readFully(DataInputStream.java:178) >> >>>>> at >> >>>>> org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.j >> >>>>>ava:64 ) at >> >>>>> org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:102 >> >>>>>) at >> >>>>> org.apache.hadoop.io.SequenceFile$Reader.readBuffer(SequenceFile.java >> >>>>>:1646) at >> >>>>> org.apache.hadoop.io.SequenceFile$Reader.seekToCurrentValue(SequenceF >> >>>>>ile.ja va:1712) at >> >>>>> org.apache.hadoop.io.SequenceFile$Reader.getCurrentValue(SequenceFile >> >>>>>.java: 1787) at >> >>>>> org.apache.hadoop.mapred.SequenceFileRecordReader.getCurrentValue(Seq >> >>>>>uenceF ileRecordReader.java:104) at >> >>>>> org.apache.hadoop.mapred.SequenceFileRecordReader.next(SequenceFileRe >> >>>>>cordRe ader.java:79) at >> >>>>> org.apache.hadoop.mapred.join.WrappedRecordReader.next(WrappedRecordR >> >>>>>eader. java:112) at >> >>>>> org.apache.hadoop.mapred.join.WrappedRecordReader.accept(WrappedRecor >> >>>>>dReade r.java:130) at >> >>>>> org.apache.hadoop.mapred.join.CompositeRecordReader.fillJoinCollector >> >>>>>(Compo siteRecordReader.java:398) at >> >>>>> org.apache.hadoop.mapred.join.JoinRecordReader.next(JoinRecordReader. >> >>>>>java:5 6) at >> >>>>> org.apache.hadoop.mapred.join.JoinRecordReader.next(JoinRecordReader. >> >>>>>java:3 3) at >> >>>>> org.apache.hadoop.mapred.MapTask$TrackedRecordReader.next(MapTask.jav >> >>>>>a:165) at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:45) at >> >>>>> org.apache.hadoop.mapred.MapTask.run(MapTask.java:227) >> >>>>> at >> >>>>> org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2209 >> >>>>>) >> >>>>> >> >>>>> >> >>>>> how can I correct this? >> >>>>> thanks. >> >>>>> Xu > >