Yongjun Zhang created HADOOP-17338: -------------------------------------- Summary: Intermittent S3AInputStream failures: Premature end of Content-Length delimited message body etc Key: HADOOP-17338 URL: https://issues.apache.org/jira/browse/HADOOP-17338 Project: Hadoop Common Issue Type: Bug Components: fs/s3 Affects Versions: 3.3.0 Reporter: Yongjun Zhang Assignee: Yongjun Zhang
We are seeing the following exceptions intermittently when using S3AInputSteam (see Symptoms at the bottom). Inspired by https://stackoverflow.com/questions/9952815/s3-java-client-fails-a-lot-with-premature-end-of-content-length-delimited-messa and https://forums.aws.amazon.com/thread.jspa?threadID=83326, we got a solution that has helped us, would like to put the fix to the community version. The problem is that S3AInputStream had a short-lived S3Object which is used to create the wrappedSteam, and this object got garbage collected and random time, which caused the stream to be closed, thus the symptoms reported. https://github.com/aws/aws-sdk-java/blob/1.11.295/aws-java-sdk-s3/src/main/java/com/amazonaws/services/s3/model/S3Object.java#L225 is the s3 code that closes the stream when S3 object is garbage collected: Here is the code in S3AInputStream that creates temporary S3Object and uses it to create the wrappedStream: {code} S3Object object = Invoker.once(text, uri, () -> client.getObject(request)); changeTracker.processResponse(object, operation, targetPos); wrappedStream = object.getObjectContent(); {code} Symptoms: 1. {code} Caused by: com.amazonaws.thirdparty.apache.http.ConnectionClosedException: Premature end of Content-Length delimited message body (expected: 156463674; received: 150001089 at com.amazonaws.thirdparty.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:178) at com.amazonaws.thirdparty.apache.http.conn.EofSensorInputStream.read(EofSensorInputStream.java:135) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.event.ProgressInputStream.read(ProgressInputStream.java:180) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.services.s3.internal.S3AbortableInputStream.read(S3AbortableInputStream.java:125) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.event.ProgressInputStream.read(ProgressInputStream.java:180) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.util.LengthCheckInputStream.read(LengthCheckInputStream.java:107) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:181) at java.io.DataInputStream.readFully(DataInputStream.java:195) at java.io.DataInputStream.readFully(DataInputStream.java:169) at org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:779) at org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:511) at org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:130) at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:214) at org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:227) at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.next(ParquetRecordReaderWrapper.java:208) at org.apache.hadoop.hive.ql.io.parquet.read.ParquetRecordReaderWrapper.next(ParquetRecordReaderWrapper.java:63) at org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader.doNext(HiveContextAwareRecordReader.java:350) ... 15 more {code} 2. {code} Caused by: javax.net.ssl.SSLException: SSL peer shut down incorrectly at sun.security.ssl.InputRecord.readV3Record(InputRecord.java:596) at sun.security.ssl.InputRecord.read(InputRecord.java:532) at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:990) at sun.security.ssl.SSLSocketImpl.readDataRecord(SSLSocketImpl.java:948) at sun.security.ssl.AppInputStream.read(AppInputStream.java:105) at com.amazonaws.thirdparty.apache.http.impl.io.SessionInputBufferImpl.streamRead(SessionInputBufferImpl.java:137) at com.amazonaws.thirdparty.apache.http.impl.io.SessionInputBufferImpl.read(SessionInputBufferImpl.java:198) at com.amazonaws.thirdparty.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:176) at com.amazonaws.thirdparty.apache.http.conn.EofSensorInputStream.read(EofSensorInputStream.java:135) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.event.ProgressInputStream.read(ProgressInputStream.java:180) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.event.ProgressInputStream.read(ProgressInputStream.java:180) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.util.LengthCheckInputStream.read(LengthCheckInputStream.java:107) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at com.amazonaws.services.s3.internal.S3AbortableInputStream.read(S3AbortableInputStream.java:125) at com.amazonaws.internal.SdkFilterInputStream.read(SdkFilterInputStream.java:82) at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:181) at java.io.DataInputStream.readFully(DataInputStream.java:195) at org.apache.hadoop.io.DataOutputBuffer$Buffer.write(DataOutputBuffer.java:70) at org.apache.hadoop.io.DataOutputBuffer.write(DataOutputBuffer.java:120) at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2361) at org.apache.hadoop.io.SequenceFile$Reader.next(SequenceFile.java:2493) at org.apache.hadoop.mapred.SequenceFileRecordReader.next(SequenceFileRecordReader.java:82) at cascading.tap.hadoop.io.CombineFileRecordReaderWrapper.next(CombineFileRecordReaderWrapper.java:70) at org.apache.hadoop.mapred.lib.CombineFileRecordReader.next(CombineFileRecordReader.java:58) at org.apache.hadoop.mapred.MapTask$TrackedRecordReader.moveToNext(MapTask.java:199) at org.apache.hadoop.mapred.MapTask$TrackedRecordReader.next(MapTask.java:185) at cascading.tap.hadoop.util.MeasuredRecordReader.next(MeasuredRecordReader.java:61) at cascading.scheme.hadoop.WritableSequenceFile.source(WritableSequenceFile.java:117) at cascading.tuple.TupleEntrySchemeIterator.getNext(TupleEntrySchemeIterator.java:163) at cascading.tuple.TupleEntrySchemeIterator.hasNext(TupleEntrySchemeIterator.java:136) ... 10 more {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: common-dev-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-dev-h...@hadoop.apache.org