Return-Path: X-Original-To: apmail-hadoop-mapreduce-user-archive@minotaur.apache.org Delivered-To: apmail-hadoop-mapreduce-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id B93389A95 for ; Fri, 30 Dec 2011 13:24:06 +0000 (UTC) Received: (qmail 26825 invoked by uid 500); 30 Dec 2011 13:24:05 -0000 Delivered-To: apmail-hadoop-mapreduce-user-archive@hadoop.apache.org Received: (qmail 26783 invoked by uid 500); 30 Dec 2011 13:24:05 -0000 Mailing-List: contact mapreduce-user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: mapreduce-user@hadoop.apache.org Delivered-To: mailing list mapreduce-user@hadoop.apache.org Received: (qmail 26775 invoked by uid 99); 30 Dec 2011 13:24:05 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 30 Dec 2011 13:24:05 +0000 X-ASF-Spam-Status: No, hits=0.7 required=5.0 tests=RCVD_IN_DNSWL_NONE,SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (athena.apache.org: local policy) Received: from [212.54.34.166] (HELO smtpq3.gn.mail.iss.as9143.net) (212.54.34.166) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 30 Dec 2011 13:23:58 +0000 Received: from [212.54.34.138] (helo=smtp7.gn.mail.iss.as9143.net) by smtpq3.gn.mail.iss.as9143.net with esmtp (Exim 4.71) (envelope-from ) id 1RgcQr-00083g-8e for mapreduce-user@hadoop.apache.org; Fri, 30 Dec 2011 14:23:37 +0100 Received: from 5249ddab.cm-4-2d.dynamic.ziggo.nl ([82.73.221.171] helo=relax.localnet) by smtp7.gn.mail.iss.as9143.net with esmtp (Exim 4.71) (envelope-from ) id 1RgcQq-0001vs-Cg for mapreduce-user@hadoop.apache.org; Fri, 30 Dec 2011 14:23:37 +0100 From: Markus Jelsma Reply-To: markus.jelsma@openindex.io Organization: Openindex To: mapreduce-user@hadoop.apache.org Subject: Re: I/O errors reading task output on 20.205.0 Date: Fri, 30 Dec 2011 14:23:05 +0100 User-Agent: KMail/1.13.5 (Linux/2.6.35-31-generic; KDE/4.5.5; i686; ; ) References: <201112270147.35524.markus.jelsma@openindex.io> <201112301354.10221.markus.jelsma@openindex.io> In-Reply-To: MIME-Version: 1.0 Content-Type: Text/Plain; charset="us-ascii" Content-Transfer-Encoding: 7bit Message-Id: <201112301423.05295.markus.jelsma@openindex.io> X-Ziggo-spambar: ---- X-Ziggo-spamscore: -4.5 X-Ziggo-spamreport: ALL_TRUSTED=-1,BAYES_00=-1.9,CM_REPLY_NOARROW=0.3,PROLO_TRUST_RDNS=-3,RDNS_DYNAMIC=0.982,TW_HD=0.077 X-Ziggo-Spam-Status: No X-Old-Spam-Flag: No X-Old-Spam-Status: No Thanks, i'll look into it! > Yes your .205 release should have it. It should fix your issue! > > On Fri, Dec 30, 2011 at 6:24 PM, Markus Jelsma > > wrote: > > Hi, (didn't reply to list before) > > > >> Does your DN log show up any form of errors when you run into this? > > > > Actually, i looked checked again to be sure and noticed errors that i > > didn't notice before: > > > > 2011-12-29 19:51:01,799 ERROR > > org.apache.hadoop.hdfs.server.datanode.DataNode: > > DatanodeRegistration(141.105.120.152:50010, > > storageID=DS-454617998-141.105.120.152-50010-1324646606851, > > infoPort=50075, ipcPort=50020):DataXceiver > > java.io.IOException: xceiverCount 258 exceeds the limit of concurrent > > xcievers 256 > > at > > org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:9 > > 2) at java.lang.Thread.run(Thread.java:662) > > > > but also this one: > > > > 2011-12-29 19:51:00,675 ERROR > > org.apache.hadoop.hdfs.server.datanode.DataNode: > > DatanodeRegistration(141.105.120.152:50010, > > storageID=DS-454617998-141.105.120.152-50010-1324646606851, > > infoPort=50075, ipcPort=50020):DataXceiver > > java.io.EOFException > > at java.io.DataInputStream.readShort(DataInputStream.java:298) > > at > > org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver > > .java:351) at > > org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:1 > > 07) at java.lang.Thread.run(Thread.java:662) > > > >> This happens with just two jobs reading how many files? And how many > >> DNs are these spread across? > > > > One file, 15 parts spread across five machines. > > > >> I'm thinking its probably something to do with your ulimits for the > >> running DN processes, but I can't say for sure without taking a look > >> at the logs. > > > > Ulimits for open files is set to 16k for all machines. > > > >> Some other stuff I can think of, a little blindly: > >> - What's your dfs.datanode.max.xcievers settings? > > > > I don't know. I increased it for a 0.22.0 test cluster but this is > > 0.20.205.0 and i haven't seen that configuration directive in the manual > > for this version. At least not in the hdfs-, core or mapred-default > > files. > > > >> - Can you ensure 'hadoop classpath' on all nodes reflects the same > >> output, and no accidental jar mixups? > > > > They are identical. All machines were installed and configured > > automatically and looking at it i don't see any differences. > > > > Is there such a max.xceivers setting in the 0.20.x branch? Judging from > > the exception it might be that's the problem. > > > > Thanks! > > > >> Does your DN log show up any form of errors when you run into this? > >> This happens with just two jobs reading how many files? And how many > >> DNs are these spread across? > >> > >> I'm thinking its probably something to do with your ulimits for the > >> running DN processes, but I can't say for sure without taking a look > >> at the logs. > >> > >> Some other stuff I can think of, a little blindly: > >> - What's your dfs.datanode.max.xcievers settings? > >> - Can you ensure 'hadoop classpath' on all nodes reflects the same > >> output, and no accidental jar mixups? > >> > >> On Thu, Dec 29, 2011 at 11:48 PM, Markus Jelsma > >> > >> wrote: > >> > We just reproduced it (somehow) by running jobs concurrently reading > >> > the same data. Two out of three similar jobs died early in the map > >> > phase with Could not obtain block errors, one finished completely. > >> > > >> > java.io.IOException: Could not obtain block: > >> > blk_119146860335302651_13067 > >> > file=/user/systems/crawl/crawldb/current/part-00000/data > >> > at > >> > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.chooseDataNode(DFSClie > >> > nt. java:2093) at > >> > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.blockSeekTo(DFSClient. > >> > jav a:1897) at > >> > org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java:20 > >> > 48) at java.io.DataInputStream.readFully(DataInputStream.java:178) at > >> > org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.ja > >> > va: 63) at > >> > org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:101) > >> > at > >> > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:1937) > >> > at > >> > org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2069) > >> > at > >> > org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader.nextKey > >> > Val ue(SequenceFileRecordReader.java:68) at > >> > org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue( > >> > Map Task.java:532) at > >> > org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67 > >> > ) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143) at > >> > org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:764) at > >> > org.apache.hadoop.mapred.MapTask.run(MapTask.java:370) at > >> > org.apache.hadoop.mapred.Child$4.run(Child.java:255) > >> > at java.security.AccessController.doPrivileged(Native Method) > >> > at javax.security.auth.Subject.doAs(Subject.java:396) > >> > at > >> > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformat > >> > ion .java:1059) at org.apache.hadoop.mapred.Child.main(Child.java:249) > >> > > >> > Another job (different but reading the same data) finished the map > >> > phase but died partially (half of the reducers) and completely > >> > frooze. > >> > > >> > 2011-12-29 18:07:58,899 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Exception in createBlockOutputStream java.io.EOFException > >> > 2011-12-29 18:07:58,899 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Abandoning block blk_4748641522370871094_13532 > >> > 2011-12-29 18:07:58,900 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Excluding datanode 141.105.120.154:50010 > >> > 2011-12-29 18:07:58,902 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Exception in createBlockOutputStream java.io.EOFException > >> > 2011-12-29 18:07:58,902 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Abandoning block blk_-1454920600140944030_13532 > >> > 2011-12-29 18:07:58,903 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Excluding datanode 141.105.120.152:50010 > >> > 2011-12-29 18:07:58,907 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Exception in createBlockOutputStream java.io.IOException: Bad connect > >> > ack with firstBadLink as 141.105.120.153:50010 > >> > 2011-12-29 18:07:58,907 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Abandoning block blk_3551418605384221738_13532 > >> > 2011-12-29 18:07:58,908 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Excluding datanode 141.105.120.153:50010 > >> > 2011-12-29 18:07:58,910 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Exception in createBlockOutputStream java.io.EOFException > >> > 2011-12-29 18:07:58,910 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Abandoning block blk_-1826030182013954555_13532 > >> > 2011-12-29 18:07:58,911 INFO org.apache.hadoop.hdfs.DFSClient: > >> > Excluding datanode 141.105.120.150:50010 > >> > 2011-12-29 18:07:58,911 WARN org.apache.hadoop.hdfs.DFSClient: > >> > DataStreamer Exception: java.io.IOException: Unable to create new > >> > block. at > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream > >> > (DF SClient.java:3213) at > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2300(DFSClient > >> > .ja va:2406) at > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSC > >> > lie nt.java:2646) > >> > > >> > 2011-12-29 18:07:58,912 WARN org.apache.hadoop.hdfs.DFSClient: Error > >> > Recovery for block blk_-1826030182013954555_13532 bad datanode[0] > >> > nodes == null 2011-12-29 18:07:58,912 WARN > >> > org.apache.hadoop.hdfs.DFSClient: Could not get block locations. > >> > Source file "/user/systems/generate- > >> > temp-1325180944829/_temporary/_attempt_201112290956_0012_r_000004_0/f > >> > etch list-13/part-00004" - Aborting... > >> > 2011-12-29 18:07:59,049 INFO > >> > org.apache.hadoop.mapred.TaskLogsTruncater: Initializing logs' > >> > truncater with mapRetainSize=-1 and > >> > reduceRetainSize=-1 2011-12-29 18:07:59,062 WARN > >> > org.apache.hadoop.mapred.Child: Error running child > >> > java.io.EOFException > >> > at java.io.DataInputStream.readShort(DataInputStream.java:298) > >> > at > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStre > >> > am( DFSClient.java:3272) at > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream > >> > (DF SClient.java:3196) at > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2300(DFSClient > >> > .ja va:2406) at > >> > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSC > >> > lie nt.java:2646) 2011-12-29 18:07:59,064 INFO > >> > org.apache.hadoop.mapred.Task: Runnning cleanup for the task > >> > > >> > It smells like the datanodes in 20.205.0 don't deal well with > >> > concurrent jobs, especially handling the same data. > >> > > >> > Is there any advice for this? Again, this does not happen on 20.203.0. > >> > Many thanks > >> > > >> >> I should add that the failing tasks that ran concurrently all read > >> >> the same map files from HDFS. > >> >> > >> >> > Hi, > >> >> > > >> >> > We just ran run large scale Apache Nutch jobs in our evaluation of > >> >> > 20.205.0 and they all failed. Some of these jobs ran concurrently > >> >> > with the fair scheduler enabled. These were simple jobs consuming > >> >> > little RAM. I double checked and there were certainly no RAM > >> >> > issues. > >> >> > > >> >> > All jobs failed and most tasks had a less than descriptive message. > >> >> > A few told they dealt with I/O errors reading task output. > >> >> > However, the data the read is fine. When we ran the same jobs > >> >> > manually (and some concurrently) some did fine and others died for > >> >> > with I/O errors reading task output again! > >> >> > > >> >> > The heap allocation for the reducers is not high but no OOM's were > >> >> > reported. Besides the occasional I/O error, which i think is > >> >> > strange enough, most tasks did not write anything to the logs that > >> >> > i can link to this problem. > >> >> > > >> >> > We do not see this happening on our 20.203.0 cluster although > >> >> > resources and settings are different. 205 is a new high-end cluster > >> >> > with similar conservative settings but only more mappers/reducers > >> >> > per node. Resource settings are almost identical. The 203 cluster > >> >> > has three times as many machines so also more open file > >> >> > descriptors and threads. > >> >> > > >> >> > Any thoughts to share? > >> >> > Thanks,