You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by stack <st...@archive.org> on 2006/03/10 20:22:13 UTC
Hung job
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)
Re: Hung job
Posted by stack <st...@archive.org>.
Whoops.
Need a means of resetting the LogFormatter loggedSevere flag when doing
a STALE_STATE soft restart of TaskTracker otherwise, TaskTracker comes
back up, checks loggedSevere, its set (still), so we restart again... ad
infinitum. Suggested patch attached.
Thanks,
St.Ack
stack wrote:
> Doug Cutting wrote:
>> stack wrote:
>>> ...
>>>
>>> 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.
>>
>> Perhaps what should happen is that the TaskTracker should exit when it
>> encounters errors reading map output.....
>>
>> I've attached a patch. The TaskTracker will restart, but with a new
>> id, so all of its tasks will be considered lost. This will
>> unfortunately lose other map tasks done by this tasktracker, but at
>> least things will keep going.
>>
>> Does this look right to you?
>>
>
> Yes. Sounds like right thing to do. Minor comments in the below.
> Meantime, let me try it.
> Thanks,
> St.Ack
>
>
>> Doug
>>
>>
> ...
>
>>
>> return 0;
>> Index: src/java/org/apache/hadoop/mapred/MapOutputFile.java
>> ===================================================================
>> --- src/java/org/apache/hadoop/mapred/MapOutputFile.java (revision
>> 385629)
>> +++ src/java/org/apache/hadoop/mapred/MapOutputFile.java (working
>> copy)
>> @@ -17,6 +17,7 @@
>> package org.apache.hadoop.mapred;
>>
>> import java.io.IOException;
>> +import java.util.logging.Level;
>>
>> import java.io.*;
>> import org.apache.hadoop.io.*;
>> @@ -108,12 +109,26 @@
>> // write the length-prefixed file content to the wire
>> File file = getOutputFile(mapTaskId, partition);
>> out.writeLong(file.length());
>> - FSDataInputStream in = FileSystem.getNamed("local",
>> this.jobConf).open(file);
>> +
>> + FSDataInputStream in = null;
>> try {
>> + in = FileSystem.getNamed("local", this.jobConf).open(file);
>> + } catch (IOException e) {
>> + // log a SEVERE exception in order to cause TaskTracker to exit
>> + TaskTracker.LOG.log(Level.SEVERE, "Can't open map output:" +
>> file, e);
>> +
>
> Should there be a 'throw e;' after TaskTracker.LOG.log above?
>
>
>
> }
>> + try {
>> byte[] buffer = new byte[8192];
>> - int l;
>> - while ((l = in.read(buffer)) != -1) {
>> + int l = 0;
>> + + while (l != -1) {
>> out.write(buffer, 0, l);
>> + try {
>> + l = in.read(buffer);
>> + } catch (IOException e) {
>> + // log a SEVERE exception in order to cause TaskTracker to
>> exit
>> + TaskTracker.LOG.log(Level.SEVERE,"Can't read map output:" +
>> file, e);
>
>
> And same here.
>
>
>> + }
>> }
>> } finally {
>> in.close();
>
Re: Hung job
Posted by Michael Stack <st...@archive.org>.
I ran overnight with the patch submitted to this list yesterday that
adds a LogFormatter.resetLoggedSevere. Twice during the night the
TaskTracker was restarted because map outputs failed checksum when
reducer came in to pick up map output parts. Each time TaskTracker came
back up... eventually. The interesting thing was that it took 9 and 12
restarts respectively as TaskTracker would restart anew because we
didn't have the map output an incoming reducer was asking for (I'm
assuming the incoming reducer has not yet been updated by jobtracker of
the new state of affairs).
This situation is a big improvement over how things used work but seems
as though we should try and avoid the TaskTracker start/stop churn.
Possibilities:
1. Add a damper so TaskTracker keeps its head down a while so its not
around when Reducer's come looking for missing map outputs, or
2. Not have map output file log severe if taskid map part being
requested is not one the TaskTracker knows about.
Neither of the above is very pretty. Any other suggestions? Otherwise
I'll look into a patch to do a variation on 2. above.
Thanks,
St.Ack
Doug Cutting wrote:
> stack wrote:
>> Yes. Sounds like right thing to do. Minor comments in the below.
>> Meantime, let me try it.
>
> Great. Please report on whether this works for you.
>
>> Should there be a 'throw e;' after TaskTracker.LOG.log above?
>
> Yes. You're right, there should be.
>
> Cheers,
>
> Doug
Re: Hung job
Posted by stack <st...@archive.org>.
Doug Cutting wrote:
> stack wrote:
>> Yes. Sounds like right thing to do. Minor comments in the below.
>> Meantime, let me try it.
>
> Great. Please report on whether this works for you.
>
>> Should there be a 'throw e;' after TaskTracker.LOG.log above?
>
> Yes. You're right, there should be.
I ran overnight and had the same issue where map output had been moved
aside because of a Checksum error -- looks like I have bad memory in one
of my slaves -- only this time, when a reducer tried to pick up the
mangled output, the tasktracker was restarted... so the patch looks like
its doing the right thing (I lost a lot of jobs unfortunately but, at
least we keep going).
Thanks,
St.Ack
>
> Cheers,
>
> Doug
Re: Hung job
Posted by Doug Cutting <cu...@apache.org>.
stack wrote:
> Yes. Sounds like right thing to do. Minor comments in the below.
> Meantime, let me try it.
Great. Please report on whether this works for you.
> Should there be a 'throw e;' after TaskTracker.LOG.log above?
Yes. You're right, there should be.
Cheers,
Doug
Re: Hung job
Posted by stack <st...@archive.org>.
Doug Cutting wrote:
> stack wrote:
>> ...
>>
>> 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.
>
> Perhaps what should happen is that the TaskTracker should exit when it
> encounters errors reading map output.....
>
> I've attached a patch. The TaskTracker will restart, but with a new id,
> so all of its tasks will be considered lost. This will unfortunately
> lose other map tasks done by this tasktracker, but at least things will
> keep going.
>
> Does this look right to you?
>
Yes. Sounds like right thing to do. Minor comments in the below.
Meantime, let me try it.
Thanks,
St.Ack
> Doug
>
>
...
>
> return 0;
> Index: src/java/org/apache/hadoop/mapred/MapOutputFile.java
> ===================================================================
> --- src/java/org/apache/hadoop/mapred/MapOutputFile.java (revision 385629)
> +++ src/java/org/apache/hadoop/mapred/MapOutputFile.java (working copy)
> @@ -17,6 +17,7 @@
> package org.apache.hadoop.mapred;
>
> import java.io.IOException;
> +import java.util.logging.Level;
>
> import java.io.*;
> import org.apache.hadoop.io.*;
> @@ -108,12 +109,26 @@
> // write the length-prefixed file content to the wire
> File file = getOutputFile(mapTaskId, partition);
> out.writeLong(file.length());
> - FSDataInputStream in = FileSystem.getNamed("local", this.jobConf).open(file);
> +
> + FSDataInputStream in = null;
> try {
> + in = FileSystem.getNamed("local", this.jobConf).open(file);
> + } catch (IOException e) {
> + // log a SEVERE exception in order to cause TaskTracker to exit
> + TaskTracker.LOG.log(Level.SEVERE, "Can't open map output:" + file, e);
> +
Should there be a 'throw e;' after TaskTracker.LOG.log above?
}
> + try {
> byte[] buffer = new byte[8192];
> - int l;
> - while ((l = in.read(buffer)) != -1) {
> + int l = 0;
> +
> + while (l != -1) {
> out.write(buffer, 0, l);
> + try {
> + l = in.read(buffer);
> + } catch (IOException e) {
> + // log a SEVERE exception in order to cause TaskTracker to exit
> + TaskTracker.LOG.log(Level.SEVERE,"Can't read map output:" + file, e);
And same here.
> + }
> }
> } finally {
> in.close();
Re: Hung job
Posted by Doug Cutting <cu...@apache.org>.
stack wrote:
> 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.
Perhaps what should happen is that the TaskTracker should exit when it
encounters errors reading map output. That way the jobtracker will
re-schedule the map, and the reduce task will wait until that map is
re-done.
I've attached a patch. The TaskTracker will restart, but with a new id,
so all of its tasks will be considered lost. This will unfortunately
lose other map tasks done by this tasktracker, but at least things will
keep going.
Does this look right to you?
Doug
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by "stack@archive.org" <st...@archive.org>.
Eric Baldeschwieler wrote:
> "Tells" is a bit strong! I just asked if this might not be so. I'm
> asking folks I know who know more. If anyone on this list does,
> please educate me!
I'm trying to get our ops-head Joerg to chime in. He has experience in
this regard.
Regards the broader picture, I've not had checksum errors over the last
3 to 4 days so have not had a chance to see what happens if the bad file
is left in place (I've been running of late on what remains of my rack
after a burnin using http://sourceforge.net/projects/va-ctcs/. It found
a bunch of issues -- particularly with RAM -- with our
less-than-top-of-the-line "commodity PCs". The rack has temporarily
been reduced to half-strength. But now I'm making progress on jobs --
no more ChecksumErrors, at least for the time being).
Thanks,
St.Ack
>
> e14
>
> On Apr 14, 2006, at 9:19 AM, Doug Cutting wrote:
>
>> Michael Stack wrote:
>>> Doug Cutting wrote:
>>>> In your case I don't think moving the files is helping at all,
>>>> since the checksum errors are not caused by bad disk blocks, but
>>>> rather by memory errors. So not moving is fine. Maybe we should
>>>> add a config parameter to disable moving on checksum error?
>>> Let me try this. Should probably subsume the
>>> 'io.skip.checksum.errors' parameter.
>>
>> Eric Baldeschwieler tells me that most disks now automatically remap
>> bad sectors (I didn't think IDE drives did this, but it seems they do
>> now). So we shouldn't need to move the files at all.
>>
>> How did things work when you disabled moving of files?
>>
>> Doug
>
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was: Re: Hung job
Posted by Eric Baldeschwieler <er...@yahoo-inc.com>.
"Tells" is a bit strong! I just asked if this might not be so. I'm
asking folks I know who know more. If anyone on this list does,
please educate me!
e14
On Apr 14, 2006, at 9:19 AM, Doug Cutting wrote:
> Michael Stack wrote:
>> Doug Cutting wrote:
>>> In your case I don't think moving the files is helping at all,
>>> since the checksum errors are not caused by bad disk blocks, but
>>> rather by memory errors. So not moving is fine. Maybe we should
>>> add a config parameter to disable moving on checksum error?
>> Let me try this. Should probably subsume the
>> 'io.skip.checksum.errors' parameter.
>
> Eric Baldeschwieler tells me that most disks now automatically
> remap bad sectors (I didn't think IDE drives did this, but it seems
> they do now). So we shouldn't need to move the files at all.
>
> How did things work when you disabled moving of files?
>
> Doug
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Doug Cutting <cu...@apache.org>.
Michael Stack wrote:
> Doug Cutting wrote:
>> In your case I don't think moving the files is helping at all, since
>> the checksum errors are not caused by bad disk blocks, but rather by
>> memory errors. So not moving is fine. Maybe we should add a config
>> parameter to disable moving on checksum error?
>
> Let me try this. Should probably subsume the 'io.skip.checksum.errors'
> parameter.
Eric Baldeschwieler tells me that most disks now automatically remap bad
sectors (I didn't think IDE drives did this, but it seems they do now).
So we shouldn't need to move the files at all.
How did things work when you disabled moving of files?
Doug
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Michael Stack <st...@archive.org>.
Doug Cutting wrote:
> ...
> The reason for the move is that, if the file lies on a bad disk block,
> we don't want to remove it, we want to keep it around so that that
> block is not reused. Local files are normally removed at the end of
> the job, but files in the bad_file directory are never automatically
> removed.
Makes sense.
>
> In your case I don't think moving the files is helping at all, since
> the checksum errors are not caused by bad disk blocks, but rather by
> memory errors. So not moving is fine. Maybe we should add a config
> parameter to disable moving on checksum error?
>
Let me try this. Should probably subsume the 'io.skip.checksum.errors'
parameter.
>> Then the task lands on a machine that has started to exhibit checksum
>> errors. After each failure, the task is rescheduled and it always
>> seems to land back at the problematic machine (Anything I can do
>> about randomizing the machine a task gets assigned too?).
>
> I think the job tracker now has logic that tries to prevent a task
> from being re-assigned to a node that it has previously failed on.
> Either that logic is buggy, or you're running an older version.
I'm using latest. Will also take a look into why tasks keep going back
to same machine.
St.Ack
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Doug Cutting <cu...@apache.org>.
Michael Stack wrote:
> What if we did not move the file? A checksum error would be thrown. If
> we're inside SequenceFile#next and 'io.skip.checksum.errors' is set,
> then we'll just try to move to next record. I do not have the
> experience with the code base to know if not-moving will manufacture
> weird scenarios elsewhere in the code base.
The reason for the move is that, if the file lies on a bad disk block,
we don't want to remove it, we want to keep it around so that that block
is not reused. Local files are normally removed at the end of the job,
but files in the bad_file directory are never automatically removed.
In your case I don't think moving the files is helping at all, since the
checksum errors are not caused by bad disk blocks, but rather by memory
errors. So not moving is fine. Maybe we should add a config parameter
to disable moving on checksum error?
> Then the task lands on a machine that has
> started to exhibit checksum errors. After each failure, the task is
> rescheduled and it always seems to land back at the problematic machine
> (Anything I can do about randomizing the machine a task gets assigned
> too?).
I think the job tracker now has logic that tries to prevent a task from
being re-assigned to a node that it has previously failed on. Either
that logic is buggy, or you're running an older version.
Doug
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Michael Stack <st...@archive.org>.
Doug Cutting wrote:
> Michael Stack wrote:
>> One question: The 'io.skip.checksum.errors' is only read in
>> SequenceFile#next but the LocalFileSystem checksum error "move-aside"
>> handler can be triggered by other than just a call out of
>> SequenceFile#next. If so, stopping the LocalFileSystem move-aside on
>> checksum error is probably not the right thing to do.
>
> Right, we ideally want SequenceFile to disable it when that flag is
> set. But that would take a lot of plumbing to implement!
Yes.
> Perhaps we should instead fix this by not closing the file in
> LocalFilesystem.reportChecksumFailure. Then it won't be able to move
> the file aside on Windows. To fix that, we can (1) try to move it
> without closing it (since something on the stack will eventually close
> it anyway, and may still need it open) and (2) if the move fails, try
> closing it and moving it (for Windows). The net effect is that
> io.skip.checksum.errors will then work on Unix but not on Windows. Or
> we could skip moving it altogether, since it seems that most checksum
> errors we're seeing are not disk errors but memory errors before the
> data hits the disk.
What if we did not move the file? A checksum error would be thrown. If
we're inside SequenceFile#next and 'io.skip.checksum.errors' is set,
then we'll just try to move to next record. I do not have the
experience with the code base to know if not-moving will manufacture
weird scenarios elsewhere in the code base.
>
> A checksum failure on a local file currently causes the task to fail.
> But it takes multiple checksum errors per job to get a job to fail,
> right? Is that what's happening?
It is. Jobs are long-running -- a day or more (I should probably try
cutting them into smaller pieces). What I usually see is a failure for
some genuinely odd reason. Then the task lands on a machine that has
started to exhibit checksum errors. After each failure, the task is
rescheduled and it always seems to land back at the problematic machine
(Anything I can do about randomizing the machine a task gets assigned too?).
St.Ack
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Doug Cutting <cu...@apache.org>.
Michael Stack wrote:
> One question: The 'io.skip.checksum.errors' is only read in
> SequenceFile#next but the LocalFileSystem checksum error "move-aside"
> handler can be triggered by other than just a call out of
> SequenceFile#next. If so, stopping the LocalFileSystem move-aside on
> checksum error is probably not the right thing to do.
Right, we ideally want SequenceFile to disable it when that flag is set.
But that would take a lot of plumbing to implement! Perhaps we should
instead fix this by not closing the file in
LocalFilesystem.reportChecksumFailure. Then it won't be able to move
the file aside on Windows. To fix that, we can (1) try to move it
without closing it (since something on the stack will eventually close
it anyway, and may still need it open) and (2) if the move fails, try
closing it and moving it (for Windows). The net effect is that
io.skip.checksum.errors will then work on Unix but not on Windows. Or
we could skip moving it altogether, since it seems that most checksum
errors we're seeing are not disk errors but memory errors before the
data hits the disk.
A checksum failure on a local file currently causes the task to fail.
But it takes multiple checksum errors per job to get a job to fail,
right? Is that what's happening? Your jobs frequently get four
checksum failures? Yikes!
Doug
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Michael Stack <st...@archive.org>.
Doug Cutting wrote:
> I think that when io.skip.checksum.errors is true, then the file
> should not be moved aside. I'd try adding code to disable the copying
> in LocalFileSystem.java when that flag is set. Does that make sense?
One question: The 'io.skip.checksum.errors' is only read in
SequenceFile#next but the LocalFileSystem checksum error "move-aside"
handler can be triggered by other than just a call out of
SequenceFile#next. If so, stopping the LocalFileSystem move-aside on
checksum error is probably not the right thing to do.
St.Ack
>
> Doug
>
> Michael Stack wrote:
>> Checksum errors seem to be a fact of life given the hardware we use.
>> They'll often cause my jobs to fail so I have been trying to figure
>> how to just skip the bad records and files. At the end is a note
>> where Stefan pointed me at 'io.skip.checksum.errors'. This
>> property, when set, triggers special handling of checksum errors
>> inside SequenceFile$Reader: If a checksum, try to skip to next
>> record. Only, this behavior can conflict with another checksum
>> handler that moves aside the problematic file whenever a checksum
>> error is found. Below is from a recent log.
>>
>> 060411 202203 task_r_22esh3 Moving bad file
>> /2/hadoop/tmp/task_r_22esh3/task_m_e3chga.out to
>> /2/bad_files/task_m_e3chga.out.1707416716
>> 060411 202203 task_r_22esh3 Bad checksum at 3578152. Skipping entries.
>> 060411 202203 task_r_22esh3 Error running child
>> 060411 202203 task_r_22esh3 java.nio.channels.ClosedChannelException
>> 060411 202203 task_r_22esh3 at
>> sun.nio.ch.FileChannelImpl.ensureOpen(FileChannelImpl.java:89)
>> 060411 202203 task_r_22esh3 at
>> sun.nio.ch.FileChannelImpl.position(FileChannelImpl.java:276)
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.fs.LocalFileSystem$LocalFSFileInputStream.seek(LocalFileSystem.java:79)
>>
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.fs.FSDataInputStream$Checker.seek(FSDataInputStream.java:67)
>>
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.fs.FSDataInputStream$PositionCache.seek(FSDataInputStream.java:164)
>>
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:193)
>>
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:243)
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.io.SequenceFile$Reader.seek(SequenceFile.java:420)
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.io.SequenceFile$Reader.sync(SequenceFile.java:431)
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.io.SequenceFile$Reader.handleChecksumException(SequenceFile.java:412)
>>
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:389)
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:209)
>> 060411 202203 task_r_22esh3 at
>> org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:709)
>>
>> (Ignore line numbers. My code is a little different from main
>> because I've other debugging code inside in SequenceFile. Otherwise
>> I'm running w/ head of hadoop).
>>
>> The SequenceFile$Reader#handleChecksumException is trying to skip to
>> next record but the file has been closed by the move-aside.
>>
>> Seems easy enough to add a 'if (this.fs.exists(new File(this.file)))'
>> to check file is still in place before we try seeking to next record.
>> I can make an issue and add a patch if wanted -- I need to test it
>> works first -- but are others failing jobs with checksum errors?
>> Thanks,
>> St.Ack
>>
>>
>>
>>
>> Michael Stack wrote:
>>
>>> 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
>>>>
>>>>
>>>>
>>>
>>
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Michael Stack <st...@archive.org>.
Doug Cutting wrote:
> I think that when io.skip.checksum.errors is true, then the file
> should not be moved aside. I'd try adding code to disable the copying
> in LocalFileSystem.java when that flag is set. Does that make sense?
>
> Doug
Let me try it over here on machines that manufacture the checksum
errors. Will file an issue and patch if it works.
Thanks,
St.Ack
Re: I get checksum errors! Was: Re: io.skip.checksum.errors was:
Re: Hung job
Posted by Doug Cutting <cu...@apache.org>.
I think that when io.skip.checksum.errors is true, then the file should
not be moved aside. I'd try adding code to disable the copying in
LocalFileSystem.java when that flag is set. Does that make sense?
Doug
Michael Stack wrote:
> Checksum errors seem to be a fact of life given the hardware we use.
> They'll often cause my jobs to fail so I have been trying to figure how
> to just skip the bad records and files. At the end is a note where
> Stefan pointed me at 'io.skip.checksum.errors'. This property, when
> set, triggers special handling of checksum errors inside
> SequenceFile$Reader: If a checksum, try to skip to next record. Only,
> this behavior can conflict with another checksum handler that moves
> aside the problematic file whenever a checksum error is found. Below is
> from a recent log.
>
> 060411 202203 task_r_22esh3 Moving bad file
> /2/hadoop/tmp/task_r_22esh3/task_m_e3chga.out to
> /2/bad_files/task_m_e3chga.out.1707416716
> 060411 202203 task_r_22esh3 Bad checksum at 3578152. Skipping entries.
> 060411 202203 task_r_22esh3 Error running child
> 060411 202203 task_r_22esh3 java.nio.channels.ClosedChannelException
> 060411 202203 task_r_22esh3 at
> sun.nio.ch.FileChannelImpl.ensureOpen(FileChannelImpl.java:89)
> 060411 202203 task_r_22esh3 at
> sun.nio.ch.FileChannelImpl.position(FileChannelImpl.java:276)
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.fs.LocalFileSystem$LocalFSFileInputStream.seek(LocalFileSystem.java:79)
>
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.fs.FSDataInputStream$Checker.seek(FSDataInputStream.java:67)
>
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.fs.FSDataInputStream$PositionCache.seek(FSDataInputStream.java:164)
>
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:193)
>
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:243)
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.io.SequenceFile$Reader.seek(SequenceFile.java:420)
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.io.SequenceFile$Reader.sync(SequenceFile.java:431)
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.io.SequenceFile$Reader.handleChecksumException(SequenceFile.java:412)
>
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:389)
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:209)
> 060411 202203 task_r_22esh3 at
> org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:709)
>
> (Ignore line numbers. My code is a little different from main because
> I've other debugging code inside in SequenceFile. Otherwise I'm running
> w/ head of hadoop).
>
> The SequenceFile$Reader#handleChecksumException is trying to skip to
> next record but the file has been closed by the move-aside.
>
> Seems easy enough to add a 'if (this.fs.exists(new File(this.file)))' to
> check file is still in place before we try seeking to next record. I can
> make an issue and add a patch if wanted -- I need to test it works first
> -- but are others failing jobs with checksum errors?
> Thanks,
> St.Ack
>
>
>
>
> Michael Stack wrote:
>
>> 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
>>>
>>>
>>>
>>
>
I get checksum errors! Was: Re: io.skip.checksum.errors was: Re:
Hung job
Posted by Michael Stack <st...@archive.org>.
Checksum errors seem to be a fact of life given the hardware we use.
They'll often cause my jobs to fail so I have been trying to figure how
to just skip the bad records and files. At the end is a note where
Stefan pointed me at 'io.skip.checksum.errors'. This property, when
set, triggers special handling of checksum errors inside
SequenceFile$Reader: If a checksum, try to skip to next record. Only,
this behavior can conflict with another checksum handler that moves
aside the problematic file whenever a checksum error is found. Below is
from a recent log.
060411 202203 task_r_22esh3 Moving bad file
/2/hadoop/tmp/task_r_22esh3/task_m_e3chga.out to
/2/bad_files/task_m_e3chga.out.1707416716
060411 202203 task_r_22esh3 Bad checksum at 3578152. Skipping entries.
060411 202203 task_r_22esh3 Error running child
060411 202203 task_r_22esh3 java.nio.channels.ClosedChannelException
060411 202203 task_r_22esh3 at
sun.nio.ch.FileChannelImpl.ensureOpen(FileChannelImpl.java:89)
060411 202203 task_r_22esh3 at
sun.nio.ch.FileChannelImpl.position(FileChannelImpl.java:276)
060411 202203 task_r_22esh3 at
org.apache.hadoop.fs.LocalFileSystem$LocalFSFileInputStream.seek(LocalFileSystem.java:79)
060411 202203 task_r_22esh3 at
org.apache.hadoop.fs.FSDataInputStream$Checker.seek(FSDataInputStream.java:67)
060411 202203 task_r_22esh3 at
org.apache.hadoop.fs.FSDataInputStream$PositionCache.seek(FSDataInputStream.java:164)
060411 202203 task_r_22esh3 at
org.apache.hadoop.fs.FSDataInputStream$Buffer.seek(FSDataInputStream.java:193)
060411 202203 task_r_22esh3 at
org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:243)
060411 202203 task_r_22esh3 at
org.apache.hadoop.io.SequenceFile$Reader.seek(SequenceFile.java:420)
060411 202203 task_r_22esh3 at
org.apache.hadoop.io.SequenceFile$Reader.sync(SequenceFile.java:431)
060411 202203 task_r_22esh3 at
org.apache.hadoop.io.SequenceFile$Reader.handleChecksumException(SequenceFile.java:412)
060411 202203 task_r_22esh3 at
org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:389)
060411 202203 task_r_22esh3 at
org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:209)
060411 202203 task_r_22esh3 at
org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:709)
(Ignore line numbers. My code is a little different from main because
I've other debugging code inside in SequenceFile. Otherwise I'm running
w/ head of hadoop).
The SequenceFile$Reader#handleChecksumException is trying to skip to
next record but the file has been closed by the move-aside.
Seems easy enough to add a 'if (this.fs.exists(new File(this.file)))' to
check file is still in place before we try seeking to next record. I can
make an issue and add a patch if wanted -- I need to test it works first
-- but are others failing jobs with checksum errors?
Thanks,
St.Ack
Michael Stack wrote:
> 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
>>
>>
>>
>
Re: io.skip.checksum.errors was: Re: Hung job
Posted by Michael Stack <st...@archive.org>.
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
>
>
>
Re: io.skip.checksum.errors was: Re: Hung job
Posted by Stefan Groschupf <sg...@media-style.com>.
Hi Stack,
I tried that but sorry to say it does not solve the problem - also
with the very latest hadoop update to nutch.
@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
io.skip.checksum.errors was: Re: Hung job
Posted by Stefan Groschupf <sg...@media-style.com>.
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