hadoop-common-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Michael Stack <st...@archive.org>
Subject Re: io.skip.checksum.errors was: Re: Hung job
Date Fri, 24 Mar 2006 01:21:13 GMT
Stefan Groschupf wrote:
> Hi Stack,
> I tried that but sorry to say it does not solve the problem - also 
> with the very latest hadoop update to nutch.
Thanks for the pointer Stefan (I only just saw this -- over-zealous spam 
filters).

Yeah, the property 'io.skip.checksum.errors' is of no use to the 
HADOOP-86 issue.  'io.skip.checksum.errors' effects SequenceFile only. 
The ChecksumException was being thrown when we had a map output file 
view on the map product rather than a sequence file view.

But 'io.skip.checksum.errors' looks like a good idea for SequenceFile.  
In fact, I just had a job fail because of the below 
'java.io.IOException: Corrupt GZIP trailer'.  It happened 4 times in a 
row.  The 'io.skip.checksum.errors' should be generalized to allow 
skipping of the likes of this exception too it seems like.

St.Ack



060323 204819 task_r_ack5c7 0.9615162% reduce > reduce
060323 204820 task_r_3rjkd7  Error running child
060323 204820 task_r_3rjkd7 java.lang.RuntimeException: 
java.io.IOException: Corrupt GZIP trailer
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:132)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.mapred.lib.IdentityReducer.reduce(IdentityReducer.java:41)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:283)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:703)
060323 204820 task_r_3rjkd7 Caused by: java.io.IOException: Corrupt GZIP 
trailer
060323 204820 task_r_3rjkd7     at 
java.util.zip.GZIPInputStream.readTrailer(GZIPInputStream.java:175)
060323 204820 task_r_3rjkd7     at 
java.util.zip.GZIPInputStream.read(GZIPInputStream.java:89)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.io.WritableUtils.readCompressedByteArray(WritableUtils.java:35)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.io.WritableUtils.readCompressedString(WritableUtils.java:70)
060323 204820 task_r_3rjkd7     at 
org.apache.nutch.parse.ParseText.readFields(ParseText.java:44)
060323 204820 task_r_3rjkd7     at 
org.apache.nutch.parse.ParseImpl.readFields(ParseImpl.java:59)
060323 204820 task_r_3rjkd7     at 
org.apache.nutch.parse.ParseImpl.read(ParseImpl.java:69)
060323 204820 task_r_3rjkd7     at 
org.apache.nutch.fetcher.FetcherOutput.readFields(FetcherOutput.java:47)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:344)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.mapred.ReduceTask$ValuesIterator.getNext(ReduceTask.java:163)
060323 204820 task_r_3rjkd7     at 
org.apache.hadoop.mapred.ReduceTask$ValuesIterator.next(ReduceTask.java:129)
060323 204820 task_r_3rjkd7     ... 3 more


>
> @Doug as far I understand the latest update already contains this 
> rollback "Reverted changes from 384385"?
>
> To summarize I'm not able to run just one larger hadoop job e.g. nutch 
> fetching never worked.
> Always one tasktracker hangs in the last 10 % of doing a reduce (some 
> times it arrived 98 % ) than the task runs into a time out and is 
> restarted again, fails, restarted, fails ...
> After 4 fails the task in progress crash the complete job.
>
> There is just nothing in the log files, that gives any hint also I 
> configured to ignore check sum exceptions.
>
> Any ideas where to search for the problem? It is very sad to throw a 
> fetched segment with several million documents just to trash,
>  since the last 3 % of reducing of one tasktracker fail.
>
> Thanks for any comments.
> Stefan
>
>
> Am 12.03.2006 um 03:33 schrieb Stefan Groschupf:
>
>> Hi Stack,
>>
>> try set the io.skip.checksum.errors to true in your hadoop-site.xml
>>
>> <property>
>>   <name>io.skip.checksum.errors</name>
>>   <value>false</value>
>>   <description>If true, when a checksum error is encountered while
>>   reading a sequence file, entries are skipped, instead of throwing an
>>   exception.</description>
>> </property>
>>
>> This may be solve your problem but I agree that there should be a 
>> smarter way than just ignoring it. :)
>>
>> Stefan
>>
>>
>> Am 10.03.2006 um 20:22 schrieb stack:
>>
>>> On hadoop-users, I've described two recent hangs.  I'm writing here 
>>> to dev because I'm looking for pointers on how best to conjure a fix 
>>> with perhaps pointers to any existing facility I might exploit (I do 
>>> not know the codebase well).
>>>
>>> In synopsis the problem goes as follows:
>>>
>>> If a reduce cannot pick up map outputs -- for example, the output 
>>> has been moved aside because of a ChecksumException (See below stack 
>>> trace) -- then the job gets stuck with the reduce task trying and 
>>> failing every ten seconds or so to pick up the non-existent map 
>>> output part.
>>>
>>> Somehow the reduce needs to give up and the jobtracker needs to 
>>> rerun the map just as it would if the tasktracker had died completely.
>>>
>>> Thanks in advance for any pointers,
>>> St.Ack
>>>
>>>
>>> 060309 014426 Moving bad file 
>>> /0/hadoop/tmp/task_m_bq2g76/part-20.out to
>>> /0/bad_files/part-20.out.2002824050
>>> 060309 014426 Server handler 0 on 50040 caught:
>>> org.apache.hadoop.fs.ChecksumException: Checksum error:
>>> /0/hadoop/tmp/task_m_bq2g76/part-20.out at 2649600
>>> org.apache.hadoop.fs.ChecksumException: Checksum error:
>>> /0/hadoop/tmp/task_m_bq2g76/part-20.out at 2649600
>>>      at
>>> org.apache.hadoop.fs.FSDataInputStream$Checker.verifySum(FSDataInputStream.java:122)

>>>
>>>      at
>>> org.apache.hadoop.fs.FSDataInputStream$Checker.read(FSDataInputStream.java:98)

>>>
>>>      at
>>> org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:158)

>>>
>>>      at java.io.BufferedInputStream.read1(BufferedInputStream.java:254)
>>>      at java.io.BufferedInputStream.read(BufferedInputStream.java:313)
>>>      at java.io.DataInputStream.read(DataInputStream.java:80)
>>>      at 
>>> org.apache.hadoop.mapred.MapOutputFile.write(MapOutputFile.java:110)
>>>      at
>>> org.apache.hadoop.io.ObjectWritable.writeObject(ObjectWritable.java:117) 
>>>
>>>      at 
>>> org.apache.hadoop.io.ObjectWritable.write(ObjectWritable.java:64)
>>>      at org.apache.hadoop.ipc.Server$Handler.run(Server.java:215)
>>>
>>>
>>>
>>
>> ---------------------------------------------
>> blog: http://www.find23.org
>> company: http://www.media-style.com
>>
>>
>>
>
> ---------------------------------------------
> blog: http://www.find23.org
> company: http://www.media-style.com
>
>
>


Mime
View raw message