[
https://issues.apache.org/jira/browse/MAPREDUCE-5767?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15641821#comment-15641821
]
Adam Ringel edited comment on MAPREDUCE-5767 at 11/7/16 8:16 PM:
-----------------------------------------------------------------
Is this possibly still a problem? I'm getting the error after a long running
map job on Hadoop 2.7.3.
I can't increase mapreduce.task.io.sort.mb > 2047 because of
[https://github.com/apache/hadoop-mapreduce/blob/HDFS-641/src/java/org/apache/hadoop/mapred/MapTask.java?source=cc#L746]
Interestingly enough some other searches from Google suggest _decreasing_ the
parameter, e.g.
[http://comments.gmane.org/gmane.comp.jakarta.lucene.hadoop.user/44152]
However, since it takes ~5hrs to reach the error point doing trial and error
testing is problematic.
---
{color:red}
UPDATE: lowering mapreduce.task.io.sort.mb to 1024 caused the problem to go
away
{color}
---
{noformat}
Task: attempt_1478217421435_0024_m_000000_0 - exited :
java.lang.ArrayIndexOutOfBoundsException
at
org.apache.hadoop.mapred.MapTask$MapOutputBuffer$Buffer.write(MapTask.java:1453)
at java.io.DataOutputStream.writeLong(DataOutputStream.java:224)
at org.apache.hadoop.io.LongWritable.write(LongWritable.java:52)
...
at
org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:98)
at
org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:82)
at
org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:1149)
at
org.apache.hadoop.mapred.MapTask$NewOutputCollector.write(MapTask.java:715)
at
org.apache.hadoop.mapreduce.task.TaskInputOutputContextImpl.write(TaskInputOutputContextImpl.java:89)
at
org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.write(WrappedMapper.java:112)
at org.apache.hadoop.mapreduce.Mapper.map(Mapper.java:125)
at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:146)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:787)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
{noformat}
was (Author: work.asr):
Is this possibly still a problem? I'm getting the error after a long running
map job on Hadoop 2.7.3.
I can't increase mapreduce.task.io.sort.mb > 2047 because of
[https://github.com/apache/hadoop-mapreduce/blob/HDFS-641/src/java/org/apache/hadoop/mapred/MapTask.java?source=cc#L746]
Interestingly enough some other searches from Google suggest _decreasing_ the
parameter, e.g.
[http://comments.gmane.org/gmane.comp.jakarta.lucene.hadoop.user/44152]
However, since it takes ~5hrs to reach the error point doing trial and error
testing is problematic.
{noformat}
Task: attempt_1478217421435_0024_m_000000_0 - exited :
java.lang.ArrayIndexOutOfBoundsException
at
org.apache.hadoop.mapred.MapTask$MapOutputBuffer$Buffer.write(MapTask.java:1453)
at java.io.DataOutputStream.writeLong(DataOutputStream.java:224)
at org.apache.hadoop.io.LongWritable.write(LongWritable.java:52)
...
at
org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:98)
at
org.apache.hadoop.io.serializer.WritableSerialization$WritableSerializer.serialize(WritableSerialization.java:82)
at
org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:1149)
at
org.apache.hadoop.mapred.MapTask$NewOutputCollector.write(MapTask.java:715)
at
org.apache.hadoop.mapreduce.task.TaskInputOutputContextImpl.write(TaskInputOutputContextImpl.java:89)
at
org.apache.hadoop.mapreduce.lib.map.WrappedMapper$Context.write(WrappedMapper.java:112)
at org.apache.hadoop.mapreduce.Mapper.map(Mapper.java:125)
at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:146)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:787)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:341)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:164)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
{noformat}
> Data corruption when single value exceeds map buffer size (io.sort.mb)
> ----------------------------------------------------------------------
>
> Key: MAPREDUCE-5767
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-5767
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Components: mrv1
> Affects Versions: 0.20.1
> Reporter: Ben Roling
>
> There is an issue in org.apache.hadoop.mapred.MapTask in 0.20 that can cause
> data corruption when the size of a single value produced by the mapper
> exceeds the size of the map output buffer (roughly io.sort.mb).
> I experienced this issue in CDH4.2.1, but am logging the issue here for
> greater visibility in case anyone else might run across the issue.
> The issue does not exist in 0.21 and beyond due to the implementation of
> MAPREDUCE-64. That JIRA significantly changes the way the map output
> buffering is done and it looks like the issue has been resolved by those
> changes.
> I expect this bug will likely be closed / won't fix due to the fact that 0.20
> is obsolete. As stated previously, I am just logging this issue for
> visibility in case anyone else is still running something based on 0.20 and
> encounters the same problem.
> In my situation the issue manifested as an ArrayIndexOutOfBoundsException in
> the reduce phase when deserializing a key -- causing the job to fail.
> However, I think the problem could manifest in a more dangerous fashion where
> the affected job succeeds, but produces corrupt output. The stack trace I
> saw was:
> 2014-02-13 01:07:34,690 WARN org.apache.hadoop.mapred.Child: Error running
> child
> java.lang.ArrayIndexOutOfBoundsException: 24
> at
> org.apache.avro.io.parsing.Symbol$Alternative.getSymbol(Symbol.java:364)
> at
> org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:229)
> at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
> at
> org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:206)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:148)
> at
> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:173)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:144)
> at
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:135)
> at
> org.apache.crunch.types.avro.SafeAvroSerialization$AvroWrapperDeserializer.deserialize(SafeAvroSerialization.java:86)
> at
> org.apache.crunch.types.avro.SafeAvroSerialization$AvroWrapperDeserializer.deserialize(SafeAvroSerialization.java:70)
> at
> org.apache.hadoop.mapreduce.task.ReduceContextImpl.nextKeyValue(ReduceContextImpl.java:135)
> at
> org.apache.hadoop.mapreduce.task.ReduceContextImpl.nextKey(ReduceContextImpl.java:114)
> at
> org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer$Context.nextKey(WrappedReducer.java:291)
> at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:163)
> at
> org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:610)
> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:444)
> at org.apache.hadoop.mapred.Child$4.run(Child.java:268)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:396)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
> at org.apache.hadoop.mapred.Child.main(Child.java:262)
> The problem appears to me to be in
> org.apache.hadoop.mapred.MapTask.MapOutputBuffer.Buffer.write(byte[], int,
> int). The sequence of events that leads up to the issue is:
> * some complete records (cumulative size less than total buffer size) written
> to buffer
> * large (over io.sort.mb) record starts writing
> * [soft buffer limit
> exceeded|https://github.com/apache/hadoop-common/blob/release-0.20.1/src/mapred/org/apache/hadoop/mapred/MapTask.java#L1030]
> - spill starts
> * write of large record continues
> * buffer becomes
> [full|https://github.com/apache/hadoop-common/blob/release-0.20.1/src/mapred/org/apache/hadoop/mapred/MapTask.java#L1012]
> *
> [wrap|https://github.com/apache/hadoop-common/blob/release-0.20.1/src/mapred/org/apache/hadoop/mapred/MapTask.java#L1013]
> evaluates to true, suggesting the buffer can be safely wrapped
> * writing the large record continues until a write occurs such that bufindex
> + len == bufstart exactly. When this happens
> [buffull|https://github.com/apache/hadoop-common/blob/release-0.20.1/src/mapred/org/apache/hadoop/mapred/MapTask.java#L1018]
> evaluates to false, so the data gets written to the buffer without event
> * writing of the large value continues with another call to write(), starting
> the corruption of the buffer. Buffer full can no longer be detected by the
> [buffull
> logic|https://github.com/apache/hadoop-common/blob/release-0.20.1/src/mapred/org/apache/hadoop/mapred/MapTask.java#L1012]
> that is used when bufindex >= bufstart
> The key to this problem occurring is a write where bufindex + len equals
> bufstart exactly.
> I have titled the issue as having to do with writing large records (over
> io.sort.mb), but really I think the issue *could* occur on smaller records if
> the serializer generated a write of exactly the right size. For example, if
> the buffer is getting close to full, but hasn't exceeded the buffer soft
> limit and then a collect() on a new value is called that triggers a write()
> such that bufindex + len == bufstart. The size of the write would have to be
> relatively large -- greater than the free space offered by the soft limit
> (20% of the buffer by default), making the issue occurring that way pretty
> unlikely.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]