You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-user@hadoop.apache.org by stack <st...@archive.org> on 2006/03/09 20:12:13 UTC

New hang (ChecksumError in map output part).

Yesterday I wrote of a hung rack. In that case -- after consultation w/
DC -- it looked like the map output files had been removed prematurely
for some unknown reason.  Subsequently, the rack is stuck spinning
eternally with reduce tasks trying and failing to pickup removed map
output parts (Looks like Stefan ran into same issue on dev list; cc'ing
Stefan in case he's not on this list).

I restarted the job with extra debugging to see if I could figure what
was provoking the premature remove of map output parts only I got a new
hang. In this case, it looks like the mapOutputServer going about
delivery of a map output part, found it to be corrupt and moved it
aside, but in a spin on yesterday's hang scenario, the remote reduce
task is stuck trying -- and failing -- to pick up the part since moved
aside.

Seems like we need to be able to get the map task rescheduled somehow
when we hit scenarios such as the one described below -- the current
hang -- or if we ever get into the hang described yesterday.  Does
mapOutputServer need to be made smarter keeping a count of failed
mapoutput pickup attempts rescheduling the map task if we hit some upper
bound?  At a minimum the job should fail?

Thanks,
St.Ack

Here is description of current hang:

Here is the map task completing 'successfully':

060309 012841 task_m_bq2g76 1.0%
rsync://narapeth003-bu.archive.org:31006/0-arc/.md5/2bf01d732ddcd6c050d2f81809d2a239.NARA-PEOT-2004-20041118220043-00250-crawling008.archive.org.arc.gz 

adding
http://ftp.arl.mil/ftp/pub/XFree86/4.1.0/binaries/Linux-ppc-glibc21/Xmod.tgz 

5805519 application/x-tar

060309 012841 Task task_m_bq2g76 is done.

Here is what looks like a Handler trying to write the part-20 over IPC,
finding its corrupt, and failing on checksum error.

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)

Thereafter, the reducer shows up ever minute or so to try and pick up
the missing part-20 and the Handler outputs the below over and over.

060309 015625 Server handler 0 on 50040 caught:
java.io.FileNotFoundException: /0/hadoop/tmp/task_m_bq2g76/part-20.out
java.io.FileNotFoundException: /0/hadoop/tmp/task_m_bq2g76/part-20.out
      at
org.apache.hadoop.fs.LocalFileSystem.openRaw(LocalFileSystem.java:113)
      at
org.apache.hadoop.fs.FSDataInputStream$Checker.<init>(FSDataInputStream.java:46)
      at
org.apache.hadoop.fs.FSDataInputStream.<init>(FSDataInputStream.java:228)
     at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:154)
      at 
org.apache.hadoop.mapred.MapOutputFile.write(MapOutputFile.java:106)
      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)

On client side, I see over and over:

060309 185013 task_r_fqlwtw copy failed: task_m_bq2g76 from
ia109317.archive.org/207.241.227.214:50040
java.io.IOException: timed out waiting for response
          at org.apache.hadoop.ipc.Client.call(Client.java:303)
          at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:141)
          at org.apache.hadoop.mapred.$Proxy2.getFile(Unknown Source)
          at
org.apache.hadoop.mapred.ReduceTaskRunner.prepare(ReduceTaskRunner.java:106)
          at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:66)
060309 185013 task_r_fqlwtw 0.24975346% reduce > copy >
task_m_bq2g76@ia109317.archive.org:50040
060309 185013 task_r_fqlwtw Got 1 map output locations.