Hey Ted, any further insights on this? We're encountering a similar
issue (on CD2). I'll be applying MAPREDUCE-1182 to see if that
resolves our case but it sounds like that JIRA didn't completely
eliminate the problem for some folks.
On Wed, Mar 10, 2010 at 11:54 PM, Ted Yu yuzhih...@gmail.com wrote:
I pressed send key a bit early.
I will have to dig a bit deeper.
Hopefully someone can find reader.close() call after which I will look for
another possible root cause :-)
On Wed, Mar 10, 2010 at 7:48 PM, Ted Yu yuzhih...@gmail.com wrote:
Thanks to Andy for the log he provided.
You can see from the log below that size increased steadily from 341535057
to 408181692, approaching maxSize. Then OOME:
2010-03-10 18:38:32,936 INFO org.apache.hadoop.mapred.ReduceTask: reserve:
pos=start requestedSize=3893000 size=341535057 numPendingRequests=0
maxSize=417601952
2010-03-10 18:38:32,936 INFO org.apache.hadoop.mapred.ReduceTask: reserve:
pos=end requestedSize=3893000 size=345428057 numPendingRequests=0
maxSize=417601952
...
2010-03-10 18:38:35,950 INFO org.apache.hadoop.mapred.ReduceTask: reserve:
pos=end requestedSize=635753 size=408181692 numPendingRequests=0
maxSize=417601952
2010-03-10 18:38:36,603 INFO org.apache.hadoop.mapred.ReduceTask: Task
attempt_201003101826_0001_r_04_0: Failed fetch #1 from
attempt_201003101826_0001_m_000875_0
2010-03-10 18:38:36,603 WARN org.apache.hadoop.mapred.ReduceTask:
attempt_201003101826_0001_r_04_0 adding host hd17.dfs.returnpath.netto
penalty box, next contact in 4 seconds
2010-03-10 18:38:36,604 INFO org.apache.hadoop.mapred.ReduceTask:
attempt_201003101826_0001_r_04_0: Got 1 map-outputs from previous
failures
2010-03-10 18:38:36,605 FATAL org.apache.hadoop.mapred.TaskRunner:
attempt_201003101826_0001_r_04_0 : Map output copy failure :
java.lang.OutOfMemoryError: Java heap space
at
org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.shuffleInMemory(ReduceTask.java:1513)
at
org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.getMapOutput(ReduceTask.java:1413)
at
org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.copyOutput(ReduceTask.java:1266)
at
org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.run(ReduceTask.java:1200)
Looking at the call to unreserve() in ReduceTask, two were for IOException
and the other was for Sanity check (line 1557). Meaning they wouldn't be
called in normal execution path.
I see one call in IFile.InMemoryReader close() method:
// Inform the RamManager
ramManager.unreserve(bufferSize);
And InMemoryReader is used in createInMemorySegments():
ReaderK, V reader =
new InMemoryReaderK, V(ramManager, mo.mapAttemptId,
mo.data, 0, mo.data.length);
But I don't see reader.close() in ReduceTask file.
On Wed, Mar 10, 2010 at 3:34 PM, Chris Douglas chri...@yahoo-inc.comwrote:
I don't think this OOM is a framework bug per se, and given the
rewrite/refactoring of the shuffle in MAPREDUCE-318 (in 0.21), tuning the
0.20 shuffle semantics is likely not worthwhile (though data informing
improvements to trunk would be excellent). Most likely (and tautologically),
ReduceTask simply requires more memory than is available and the job failure
can be avoided by either 0) increasing the heap size or 1) lowering
mapred.shuffle.input.buffer.percent. Most of the tasks we run have a heap of
1GB. For a reduce fetching 200k map outputs, that's a reasonable, even
stingy amount of space. -C
On Mar 10, 2010, at 5:26 AM, Ted Yu wrote:
I verified that size and maxSize are long. This means MR-1182 didn't
resolve
Andy's issue.
According to Andy:
At the beginning of the job there are 209,754 pending map tasks and 32
pending reduce tasks
My guess is that GC wasn't reclaiming memory fast enough, leading to OOME
because of large number of in-memory shuffle candidates.
My suggestion for Andy would be to:
1. add -*verbose*:*gc as JVM parameter
2. modify reserve() slightly to calculate the maximum outstanding
numPendingRequests and print the maximum.
Based on the output from above two items, we can discuss solution.
My intuition is to place upperbound on numPendingRequests beyond which
canFitInMemory() returns false.
*
My two cents.
On Tue, Mar 9, 2010 at 11:51 PM, Christopher Douglas
chri...@yahoo-inc.comwrote:
That section of code is unmodified in MR-1182. See the patches/svn log.
-C
Sent from my iPhone
On Mar 9, 2010, at 7:44 PM, Ted Yu yuzhih...@gmail.com wrote:
I just downloaded hadoop-0.20.2 tar ball from cloudera mirror.
This is what I see in ReduceTask (line 999):
public synchronized boolean reserve(int requestedSize, InputStream
in)
throws InterruptedException {
// Wait till the request can be fulfilled...
while ((size + requestedSize) maxSize) {
I don't see the fix from MR-1182.