Steve, thanks for the reply.  Digging through all the documentation now.

Much appreciated!



On 16 May 2017 at 10:10, Steve Loughran <ste...@hortonworks.com> wrote:

>
> On 11 May 2017, at 06:07, lucas.g...@gmail.com wrote:
>
> Hi users, we have a bunch of pyspark jobs that are using S3 for loading /
> intermediate steps and final output of parquet files.
>
>
> Please don't, not without a committer specially written to work against S3
> in the presence of failures.You are at risk of things going wrong and you
> not even noticing.
>
> The only one that I trust to do this right now is;
> https://github.com/rdblue/s3committer
>
>
> see also : https://github.com/apache/spark/blob/master/docs/cloud-
> integration.md
>
>
>
> We're running into the following issues on a semi regular basis:
> * These are intermittent errors, IE we have about 300 jobs that run
> nightly... And a fairly random but small-ish percentage of them fail with
> the following classes of errors.
>
>
> *S3 write errors *
>
>> "ERROR Utils: Aborting task
>> com.amazonaws.services.s3.model.AmazonS3Exception: Status Code: 404, AWS
>> Service: Amazon S3, AWS Request ID: 2D3RP, AWS Error Code: null, AWS Error
>> Message: Not Found, S3 Extended Request ID: BlaBlahEtc="
>>
>
>
>> "Py4JJavaError: An error occurred while calling o43.parquet.
>> : com.amazonaws.services.s3.model.MultiObjectDeleteException: Status
>> Code: 0, AWS Service: null, AWS Request ID: null, AWS Error Code: null, AWS
>> Error Message: One or more objects could not be deleted, S3 Extended
>> Request ID: null"
>
>
>
>
> *S3 Read Errors: *
>
>> [Stage 1:=================================================>       (27 +
>> 4) / 31]17/05/10 16:25:23 ERROR Executor: Exception in task 10.0 in stage
>> 1.0 (TID 11)
>> java.net.SocketException: Connection reset
>> at java.net.SocketInputStream.read(SocketInputStream.java:196)
>> at java.net.SocketInputStream.read(SocketInputStream.java:122)
>> at sun.security.ssl.InputRecord.readFully(InputRecord.java:442)
>> at sun.security.ssl.InputRecord.readV3Record(InputRecord.java:554)
>> at sun.security.ssl.InputRecord.read(InputRecord.java:509)
>> at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:927)
>> at sun.security.ssl.SSLSocketImpl.readDataRecord(SSLSocketImpl.java:884)
>> at sun.security.ssl.AppInputStream.read(AppInputStream.java:102)
>> at org.apache.http.impl.io.AbstractSessionInputBuffer.read(
>> AbstractSessionInputBuffer.java:198)
>> at org.apache.http.impl.io.ContentLengthInputStream.read(
>> ContentLengthInputStream.java:178)
>> at org.apache.http.impl.io.ContentLengthInputStream.read(
>> ContentLengthInputStream.java:200)
>> at org.apache.http.impl.io.ContentLengthInputStream.close(
>> ContentLengthInputStream.java:103)
>> at org.apache.http.conn.BasicManagedEntity.streamClosed(
>> BasicManagedEntity.java:168)
>> at org.apache.http.conn.EofSensorInputStream.checkClose(
>> EofSensorInputStream.java:228)
>> at org.apache.http.conn.EofSensorInputStream.close(
>> EofSensorInputStream.java:174)
>> at java.io.FilterInputStream.close(FilterInputStream.java:181)
>> at java.io.FilterInputStream.close(FilterInputStream.java:181)
>> at java.io.FilterInputStream.close(FilterInputStream.java:181)
>> at java.io.FilterInputStream.close(FilterInputStream.java:181)
>> at com.amazonaws.services.s3.model.S3Object.close(S3Object.java:203)
>> at org.apache.hadoop.fs.s3a.S3AInputStream.close(S3AInputStream.java:187)
>
>
>
> We have literally tons of logs we can add but it would make the email
> unwieldy big.  If it would be helpful I'll drop them in a pastebin or
> something.
>
> Our config is along the lines of:
>
>    - spark-2.1.0-bin-hadoop2.7
>    - '--packages 
> com.amazonaws:aws-java-sdk:1.10.34,org.apache.hadoop:hadoop-aws:2.6.0
>    pyspark-shell'
>
>
> You should have the Hadoop 2.7 JARs on your CP, as s3a on 2.6 wasn't ready
> to play with. In particular, in a close() call it reads to the end of the
> stream, which is a performance killer on large files. That stack trace you
> see is from that same phase of operation, so should go away too.
>
> Hadoop 2.7.3 depends on Amazon SDK 1.7.4; trying to use a different one
> will probably cause link errors.
> http://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws/2.7.3
>
> Also: make sure Joda time >= 2.8.1 for Java 8
>
> If you go up to 2.8.0, and you still see the errors, file something
> against HADOOP in JIRA
>
>
> Given the stack overflow / googling I've been doing I know we're not the
> only org with these issues but I haven't found a good set of solutions in
> those spaces yet.
>
> Thanks!
>
> Gary Lucas
>
>
>

Reply via email to