[
https://issues.apache.org/jira/browse/MAPREDUCE-115?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Joydeep Sen Sarma reopened MAPREDUCE-115:
-----------------------------------------
re-opening. we are seeing this a lot on hadoop-20 (yahoo distribution):
1. reducers not able to fetch map outputs because map side tasktracker cannot
locate map output
2. mappers not able to locate previously spilled data
Scott has added logging that is telling us that:
- for #1. that the map output file was actually present/created at the time the
map was first reported to be done
- that we have not removed the mapoutput file (from the TT code path deleting
the files) before the reducer fetch request came in
so something very fishy - seems like either the files disappear in the interim
- or that the localdirallocator is not being able to find things that are
actually present.
> Map tasks are receiving FileNotFound Exceptions for spill files on a regular
> basis and are getting killed
> ---------------------------------------------------------------------------------------------------------
>
> Key: MAPREDUCE-115
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-115
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Reporter: Jothi Padmanabhan
>
> The following is the log -- Map tasks are unable to locate the spill files
> when they are doing the final merge (mergeParts).
> java.io.FileNotFoundException: File
> /xxx/mapred-tt/mapred-local/taskTracker/jobcache/job_200808190959_0001/attempt_200808190959_0001_m_000000_0/output/spill23.out
> does not exist.
> at
> org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:420)
> at
> org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:244)
> at
> org.apache.hadoop.fs.FileSystem.getContentSummary(FileSystem.java:682)
> at
> org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.getFileLength(ChecksumFileSystem.java:218)
> at
> org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.seek(ChecksumFileSystem.java:259)
> at
> org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:37)
> at
> org.apache.hadoop.mapred.MapTask$MapOutputBuffer.mergeParts(MapTask.java:1102)
> at
> org.apache.hadoop.mapred.MapTask$MapOutputBuffer.flush(MapTask.java:769)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:255)
> at
> org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2208)
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.