You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Takenori Sato (JIRA)" <ji...@apache.org> on 2015/03/19 04:20:39 UTC

[jira] [Reopened] (HADOOP-10037) s3n read truncated, but doesn't throw exception

     [ https://issues.apache.org/jira/browse/HADOOP-10037?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Takenori Sato reopened HADOOP-10037:
------------------------------------

I confirmed this happens on Hadoop 2.6.0, and found the reason.

Here's the stacktrace.

{quote}

2015-03-13 20:17:24,866 [TezChild] DEBUG org.jets3t.service.impl.rest.httpclient.HttpMethodReleaseInputStream - Released HttpMethod as its response data stream threw an exception
org.apache.http.ConnectionClosedException: Premature end of Content-Length delimited message body (expected: 296587138; received: 155648
	at org.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:184)
	at org.apache.http.conn.EofSensorInputStream.read(EofSensorInputStream.java:138)
	at org.jets3t.service.io.InterruptableInputStream.read(InterruptableInputStream.java:78)
	at org.jets3t.service.impl.rest.httpclient.HttpMethodReleaseInputStream.read(HttpMethodReleaseInputStream.java:146)
	at org.apache.hadoop.fs.s3native.NativeS3FileSystem$NativeS3FsInputStream.read(NativeS3FileSystem.java:145)
	at java.io.BufferedInputStream.read1(BufferedInputStream.java:273)
	at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
	at java.io.DataInputStream.read(DataInputStream.java:100)
	at org.apache.hadoop.util.LineReader.fillBuffer(LineReader.java:180)
	at org.apache.hadoop.util.LineReader.readDefaultLine(LineReader.java:216)
	at org.apache.hadoop.util.LineReader.readLine(LineReader.java:174)
	at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.nextKeyValue(LineRecordReader.java:185)
	at org.apache.pig.builtin.PigStorage.getNext(PigStorage.java:259)
	at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigRecordReader.nextKeyValue(PigRecordReader.java:204)
	at org.apache.tez.mapreduce.lib.MRReaderMapReduce.next(MRReaderMapReduce.java:116)
	at org.apache.pig.backend.hadoop.executionengine.tez.plan.operator.POSimpleTezLoad.getNextTuple(POSimpleTezLoad.java:106)
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:307)
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach.getNextTuple(POForEach.java:246)
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:307)
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter.getNextTuple(POFilter.java:91)
	at org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator.processInput(PhysicalOperator.java:307)
	at org.apache.pig.backend.hadoop.executionengine.tez.plan.operator.POStoreTez.getNextTuple(POStoreTez.java:117)
	at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.runPipeline(PigProcessor.java:313)
	at org.apache.pig.backend.hadoop.executionengine.tez.runtime.PigProcessor.run(PigProcessor.java:192)
	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:415)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
2015-03-13 20:17:24,867 [TezChild] INFO  org.apache.hadoop.fs.s3native.NativeS3FileSystem - Received IOException while reading 'user/hadoop/tsato/readlarge/input/cloudian-s3.log.20141119', attempting to reopen.
2015-03-13 20:17:24,867 [TezChild] DEBUG org.jets3t.service.impl.rest.httpclient.HttpMethodReleaseInputStream - Released HttpMethod as its response data stream is fully consumed
2015-03-13 20:17:24,868 [TezChild] INFO  org.apache.tez.dag.app.TaskAttemptListenerImpTezDag - Commit go/no-go request from attempt_1426245338920_0001_1_00_000004_0
2015-03-13 20:17:24,868 [TezChild] INFO  org.apache.tez.dag.app.dag.impl.TaskImpl - attempt_1426245338920_0001_1_00_000004_0 given a go for committing the task output.

{quote}

The problem is that a job successfully finishes after the exception. Thus, its output gets truncated.

The cause is in the retry logic:

{code:title=NativeS3FileSystem#read(byte[] b, int off, int len)|borderStyle=solid}
      int result = -1;
      try {
        result = in.read(b, off, len);
      } catch (EOFException eof) {
        throw eof;
      } catch (IOException e) {
        LOG.info( "Received IOException while reading '{}'," +
                  " attempting to reopen.", key);
        seek(pos);
        result = in.read(b, off, len);
      }
{code}

After catching the IOException, it attempts to read again from the same inputstream. _NativeS3FileSystem#seek()_ does nothing because the position is not changed. Then, the successive _NativeS3FileSystem#read()_ returns -1 instead of throwing another exception to abort the job. It is done by *EofSensorInputStream* because _EofSensorInputStream#checkAbort()_ makes EofSensorInputStream return false from _EofSensorInputStream#isReadAllowed()_.. 

{code:title=EofSensorInputStream#read(byte[] b, int off, int len)|borderStyle=solid}
    public int read() throws IOException {
        int l = -1;

        if (isReadAllowed()) {
            try {
                l = wrappedStream.read();
                checkEOF(l);
            } catch (IOException ex) {
                checkAbort();
                throw ex;
            }
        }

        return l;
    }
{code}

I didn't reach to the history why such a retry is attempted. Even If it does so, I think the inner inputstream should be carefully updated.

If that's not obvious today, I think making it simply fail without retry is reasonable.

> s3n read truncated, but doesn't throw exception 
> ------------------------------------------------
>
>                 Key: HADOOP-10037
>                 URL: https://issues.apache.org/jira/browse/HADOOP-10037
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 2.0.0-alpha
>         Environment: Ubuntu Linux 13.04 running on Amazon EC2 (cc2.8xlarge)
>            Reporter: David Rosenstrauch
>             Fix For: 2.6.0
>
>         Attachments: S3ReadFailedOnTruncation.html, S3ReadSucceeded.html
>
>
> For months now we've been finding that we've been experiencing frequent data truncation issues when reading from S3 using the s3n:// protocol.  I finally was able to gather some debugging output on the issue in a job I ran last night, and so can finally file a bug report.
> The job I ran last night was on a 16-node cluster (all of them AWS EC2 cc2.8xlarge machines, running Ubuntu 13.04 and Cloudera CDH4.3.0).  The job was a Hadoop streaming job, which reads through a large number (i.e., ~55,000) of files on S3, each of them approximately 300K bytes in size.
> All of the files contain 46 columns of data in each record.  But I added in an extra check in my mapper code to count and verify the number of columns in every record - throwing an error and crashing the map task if the column count is wrong.
> If you look in the attached task logs, you'll see 2 attempts on the same task.  The first one fails due to data truncated (i.e., my job intentionally fails the map task due to the current record failing the column count check).  The task then gets retried on a different machine and runs to a succesful completion.
> You can see further evidence of the truncation further down in the task logs, where it displays the count of the records read:  the failed task says 32953 records read, while the successful task says 63133.
> Any idea what the problem might be here and/or how to work around it?  This issue is a very common occurrence on our clusters.  E.g., in the job I ran last night before I had gone to bed I had already encountered 8 such failuers, and the job was only 10% complete.  (~25,000 out of ~250,000 tasks.)
> I realize that it's common for I/O errors to occur - possibly even frequently - in a large Hadoop job.  But I would think that if an I/O failure (like a truncated read) did occur, that something in the underlying infrastructure code (i.e., either in NativeS3FileSystem or in jets3t) should detect the error and throw an IOException accordingly.  It shouldn't be up to the calling code to detect such failures, IMO.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)