You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by "lucas.gary@gmail.com" <lu...@gmail.com> on 2017/05/11 05:07:21 UTC

Spark <--> S3 flakiness

Hi users, we have a bunch of pyspark jobs that are using S3 for loading /
intermediate steps and final output of parquet files.

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'

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

Re: Spark <--> S3 flakiness

Posted by Steve Loughran <st...@hortonworks.com>.
On 18 May 2017, at 05:29, lucas.gary@gmail.com<ma...@gmail.com> wrote:

Steve, just to clarify:

"FWIW, if you can move up to the Hadoop 2.8 version of the S3A client it is way better on high-performance reads, especially if you are working with column data and can set the fs.s3a.experimental.fadvise=random option. "

Are you talking about the hadoop-aws lib or hadoop itself.  I see that spark is currently only pre-built against hadoop 2.7.

all the hadoop JARs. It's a big move, and really I'd hold off for it except in the special case : spark standalone on my desktop

Most of our failures are on write, the other fix I've seen advertised has been: "fileoutputcommitter.algorithm.version=2"

this eliminates the big rename() in job commit, renaming the work of individual tasks at the end of each task commit.

It doesn't do anything for problems writing data, and it still has a fundamental flaw: to rename everything in a "directory tree", you need to be able to list all objects under a path, which utterly depends on consistent directory listings. Amazon S3 doesn't offer that: you can create a file, then list the bucket *and not see the file*. Similarly, after deletion it may be listed, but not be there any more. Without that consistent listing, you don't get reliable renames, hence output.

It's possible that you may not even notice the fact that data hasn't been copied over.

Ryan's committer avoids this problem by using the local filesystem and HDFS cluster as the consistent stores, and using uncompleted S3A multipart uploads to eliminate the rename at the end

https://github.com/rdblue/s3committer

see also: https://www.youtube.com/watch?v=8F2Jqw5_OnI&feature=youtu.be



Still doing some reading and will start testing in the next day or so.

Thanks!

Gary

On 17 May 2017 at 03:19, Steve Loughran <st...@hortonworks.com>> wrote:

On 17 May 2017, at 06:00, lucas.gary@gmail.com<ma...@gmail.com> wrote:

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

Much appreciated!



FWIW, if you can move up to the Hadoop 2.8 version of the S3A client it is way better on high-performance reads, especially if you are working with column data and can set the fs.s3a.experimental.fadvise=random option.

That's in apache Hadoop 2.8, HDP 2.5+, and I suspect also the latest versions of CDH, even if their docs don't mention it

https://hortonworks.github.io/hdp-aws/s3-performance/
https://www.cloudera.com/documentation/enterprise/5-9-x/topics/spark_s3.html


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

On 11 May 2017, at 06:07, lucas.gary@gmail.com<ma...@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<http://org.apache.http.impl.io/>.AbstractSessionInputBuffer.read(AbstractSessionInputBuffer.java:198)
at org.apache.http.impl.io<http://org.apache.http.impl.io/>.ContentLengthInputStream.read(ContentLengthInputStream.java:178)
at org.apache.http.impl.io<http://org.apache.http.impl.io/>.ContentLengthInputStream.read(ContentLengthInputStream.java:200)
at org.apache.http.impl.io<http://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






Re: Spark <--> S3 flakiness

Posted by "lucas.gary@gmail.com" <lu...@gmail.com>.
Steve, just to clarify:

"FWIW, if you can move up to the Hadoop 2.8 version of the S3A client it is
way better on high-performance reads, especially if you are working with
column data and can set the fs.s3a.experimental.fadvise=random option. "

Are you talking about the hadoop-aws lib or hadoop itself.  I see that
spark is currently only pre-built against hadoop 2.7.

Most of our failures are on write, the other fix I've seen advertised has
been: "fileoutputcommitter.algorithm.version=2"

Still doing some reading and will start testing in the next day or so.

Thanks!

Gary

On 17 May 2017 at 03:19, Steve Loughran <st...@hortonworks.com> wrote:

>
> On 17 May 2017, at 06:00, lucas.gary@gmail.com wrote:
>
> Steve, thanks for the reply.  Digging through all the documentation now.
>
> Much appreciated!
>
>
>
> FWIW, if you can move up to the Hadoop 2.8 version of the S3A client it is
> way better on high-performance reads, especially if you are working with
> column data and can set the fs.s3a.experimental.fadvise=random option.
>
> That's in apache Hadoop 2.8, HDP 2.5+, and I suspect also the latest
> versions of CDH, even if their docs don't mention it
>
> https://hortonworks.github.io/hdp-aws/s3-performance/
> https://www.cloudera.com/documentation/enterprise/5-9-
> x/topics/spark_s3.html
>
>
> On 16 May 2017 at 10:10, Steve Loughran <st...@hortonworks.com> wrote:
>
>>
>> On 11 May 2017, at 06:07, lucas.gary@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-int
>> egration.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(Abst
>>> ractSessionInputBuffer.java:198)
>>> at org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>>> tLengthInputStream.java:178)
>>> at org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>>> tLengthInputStream.java:200)
>>> at org.apache.http.impl.io.ContentLengthInputStream.close(Conte
>>> ntLengthInputStream.java:103)
>>> at org.apache.http.conn.BasicManagedEntity.streamClosed(BasicMa
>>> nagedEntity.java:168)
>>> at org.apache.http.conn.EofSensorInputStream.checkClose(EofSens
>>> orInputStream.java:228)
>>> at org.apache.http.conn.EofSensorInputStream.close(EofSensorInp
>>> utStream.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.1
>>    0.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
>>
>>
>>
>
>

Re: Spark <--> S3 flakiness

Posted by Steve Loughran <st...@hortonworks.com>.
On 17 May 2017, at 06:00, lucas.gary@gmail.com<ma...@gmail.com> wrote:

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

Much appreciated!



FWIW, if you can move up to the Hadoop 2.8 version of the S3A client it is way better on high-performance reads, especially if you are working with column data and can set the fs.s3a.experimental.fadvise=random option.

That's in apache Hadoop 2.8, HDP 2.5+, and I suspect also the latest versions of CDH, even if their docs don't mention it

https://hortonworks.github.io/hdp-aws/s3-performance/
https://www.cloudera.com/documentation/enterprise/5-9-x/topics/spark_s3.html


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

On 11 May 2017, at 06:07, lucas.gary@gmail.com<ma...@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<http://org.apache.http.impl.io/>.AbstractSessionInputBuffer.read(AbstractSessionInputBuffer.java:198)
at org.apache.http.impl.io<http://org.apache.http.impl.io/>.ContentLengthInputStream.read(ContentLengthInputStream.java:178)
at org.apache.http.impl.io<http://org.apache.http.impl.io/>.ContentLengthInputStream.read(ContentLengthInputStream.java:200)
at org.apache.http.impl.io<http://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




Re: Spark <--> S3 flakiness

Posted by "lucas.gary@gmail.com" <lu...@gmail.com>.
Steve, thanks for the reply.  Digging through all the documentation now.

Much appreciated!



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

>
> On 11 May 2017, at 06:07, lucas.gary@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
>
>
>

Re: Spark <--> S3 flakiness

Posted by Steve Loughran <st...@hortonworks.com>.
On 11 May 2017, at 06:07, lucas.gary@gmail.com<ma...@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


Re: Spark <--> S3 flakiness

Posted by Gourav Sengupta <go...@gmail.com>.
Are you running EMR?

On Sun, May 14, 2017 at 4:59 AM, Miguel Morales <th...@gmail.com>
wrote:

> Some things just didn't work as i had first expected it.  For example,
> when writing from a spark collection to an alluxio destination didn't
> persist them to s3 automatically.
>
> I remember having to use the alluxio library directly to force the
> files to persist to s3 after spark finished writing to alluxio.
>
> On Fri, May 12, 2017 at 6:52 AM, Gene Pang <ge...@gmail.com> wrote:
> > Hi,
> >
> > Yes, you can use Alluxio with Spark to read/write to S3. Here is a blog
> post
> > on Spark + Alluxio + S3, and here is some documentation for configuring
> > Alluxio + S3 and configuring Spark + Alluxio.
> >
> > You mentioned that it required a lot of effort to get working. May I ask
> > what you ran into, and how you got it to work?
> >
> > Thanks,
> > Gene
> >
> > On Thu, May 11, 2017 at 11:55 AM, Miguel Morales <
> therevoltingx@gmail.com>
> > wrote:
> >>
> >> Might want to try to use gzip as opposed to parquet.  The only way i
> >> ever reliably got parquet to work on S3 is by using Alluxio as a
> >> buffer, but it's a decent amount of work.
> >>
> >> On Thu, May 11, 2017 at 11:50 AM, lucas.gary@gmail.com
> >> <lu...@gmail.com> wrote:
> >> > Also, and this is unrelated to the actual question... Why don't these
> >> > messages show up in the archive?
> >> >
> >> > http://apache-spark-user-list.1001560.n3.nabble.com/
> >> >
> >> > Ideally I'd want to post a link to our internal wiki for these
> >> > questions,
> >> > but can't find them in the archive.
> >> >
> >> > On 11 May 2017 at 07:16, lucas.gary@gmail.com <lu...@gmail.com>
> >> > wrote:
> >> >>
> >> >> Looks like this isn't viable in spark 2.0.0 (and greater I presume).
> >> >> I'm
> >> >> pretty sure I came across this blog and ignored it due to that.
> >> >>
> >> >> Any other thoughts?  The linked tickets in:
> >> >> https://issues.apache.org/jira/browse/SPARK-10063
> >> >> https://issues.apache.org/jira/browse/HADOOP-13786
> >> >> https://issues.apache.org/jira/browse/HADOOP-9565 look relevant too.
> >> >>
> >> >> On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com>
> >> >> wrote:
> >> >>>
> >> >>> Try using the DirectParquetOutputCommiter:
> >> >>> http://dev.sortable.com/spark-directparquetoutputcommitter/
> >> >>>
> >> >>> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
> >> >>> <lu...@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.
> >> >>> >
> >> >>> > 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'
> >> >>> >
> >> >>> > 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
> >> >>
> >> >>
> >> >
> >>
> >> ---------------------------------------------------------------------
> >> To unsubscribe e-mail: user-unsubscribe@spark.apache.org
> >>
> >
>
> ---------------------------------------------------------------------
> To unsubscribe e-mail: user-unsubscribe@spark.apache.org
>
>

Re: Spark <--> S3 flakiness

Posted by Miguel Morales <th...@gmail.com>.
Some things just didn't work as i had first expected it.  For example,
when writing from a spark collection to an alluxio destination didn't
persist them to s3 automatically.

I remember having to use the alluxio library directly to force the
files to persist to s3 after spark finished writing to alluxio.

On Fri, May 12, 2017 at 6:52 AM, Gene Pang <ge...@gmail.com> wrote:
> Hi,
>
> Yes, you can use Alluxio with Spark to read/write to S3. Here is a blog post
> on Spark + Alluxio + S3, and here is some documentation for configuring
> Alluxio + S3 and configuring Spark + Alluxio.
>
> You mentioned that it required a lot of effort to get working. May I ask
> what you ran into, and how you got it to work?
>
> Thanks,
> Gene
>
> On Thu, May 11, 2017 at 11:55 AM, Miguel Morales <th...@gmail.com>
> wrote:
>>
>> Might want to try to use gzip as opposed to parquet.  The only way i
>> ever reliably got parquet to work on S3 is by using Alluxio as a
>> buffer, but it's a decent amount of work.
>>
>> On Thu, May 11, 2017 at 11:50 AM, lucas.gary@gmail.com
>> <lu...@gmail.com> wrote:
>> > Also, and this is unrelated to the actual question... Why don't these
>> > messages show up in the archive?
>> >
>> > http://apache-spark-user-list.1001560.n3.nabble.com/
>> >
>> > Ideally I'd want to post a link to our internal wiki for these
>> > questions,
>> > but can't find them in the archive.
>> >
>> > On 11 May 2017 at 07:16, lucas.gary@gmail.com <lu...@gmail.com>
>> > wrote:
>> >>
>> >> Looks like this isn't viable in spark 2.0.0 (and greater I presume).
>> >> I'm
>> >> pretty sure I came across this blog and ignored it due to that.
>> >>
>> >> Any other thoughts?  The linked tickets in:
>> >> https://issues.apache.org/jira/browse/SPARK-10063
>> >> https://issues.apache.org/jira/browse/HADOOP-13786
>> >> https://issues.apache.org/jira/browse/HADOOP-9565 look relevant too.
>> >>
>> >> On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com>
>> >> wrote:
>> >>>
>> >>> Try using the DirectParquetOutputCommiter:
>> >>> http://dev.sortable.com/spark-directparquetoutputcommitter/
>> >>>
>> >>> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
>> >>> <lu...@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.
>> >>> >
>> >>> > 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'
>> >>> >
>> >>> > 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
>> >>
>> >>
>> >
>>
>> ---------------------------------------------------------------------
>> To unsubscribe e-mail: user-unsubscribe@spark.apache.org
>>
>

---------------------------------------------------------------------
To unsubscribe e-mail: user-unsubscribe@spark.apache.org


Re: Spark <--> S3 flakiness

Posted by Gene Pang <ge...@gmail.com>.
Hi,

Yes, you can use Alluxio with Spark to read/write to S3. Here is a blog
post on Spark + Alluxio + S3
<https://www.alluxio.com/blog/accelerating-on-demand-data-analytics-with-alluxio>,
and here is some documentation for configuring Alluxio + S3
<http://www.alluxio.org/docs/master/en/Configuring-Alluxio-with-S3.html>
and configuring Spark + Alluxio
<http://www.alluxio.org/docs/master/en/Running-Spark-on-Alluxio.html>.

You mentioned that it required a lot of effort to get working. May I ask
what you ran into, and how you got it to work?

Thanks,
Gene

On Thu, May 11, 2017 at 11:55 AM, Miguel Morales <th...@gmail.com>
wrote:

> Might want to try to use gzip as opposed to parquet.  The only way i
> ever reliably got parquet to work on S3 is by using Alluxio as a
> buffer, but it's a decent amount of work.
>
> On Thu, May 11, 2017 at 11:50 AM, lucas.gary@gmail.com
> <lu...@gmail.com> wrote:
> > Also, and this is unrelated to the actual question... Why don't these
> > messages show up in the archive?
> >
> > http://apache-spark-user-list.1001560.n3.nabble.com/
> >
> > Ideally I'd want to post a link to our internal wiki for these questions,
> > but can't find them in the archive.
> >
> > On 11 May 2017 at 07:16, lucas.gary@gmail.com <lu...@gmail.com>
> wrote:
> >>
> >> Looks like this isn't viable in spark 2.0.0 (and greater I presume).
> I'm
> >> pretty sure I came across this blog and ignored it due to that.
> >>
> >> Any other thoughts?  The linked tickets in:
> >> https://issues.apache.org/jira/browse/SPARK-10063
> >> https://issues.apache.org/jira/browse/HADOOP-13786
> >> https://issues.apache.org/jira/browse/HADOOP-9565 look relevant too.
> >>
> >> On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com>
> wrote:
> >>>
> >>> Try using the DirectParquetOutputCommiter:
> >>> http://dev.sortable.com/spark-directparquetoutputcommitter/
> >>>
> >>> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
> >>> <lu...@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.
> >>> >
> >>> > 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'
> >>> >
> >>> > 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
> >>
> >>
> >
>
> ---------------------------------------------------------------------
> To unsubscribe e-mail: user-unsubscribe@spark.apache.org
>
>

Re: Spark <--> S3 flakiness

Posted by Miguel Morales <th...@gmail.com>.
Might want to try to use gzip as opposed to parquet.  The only way i
ever reliably got parquet to work on S3 is by using Alluxio as a
buffer, but it's a decent amount of work.

On Thu, May 11, 2017 at 11:50 AM, lucas.gary@gmail.com
<lu...@gmail.com> wrote:
> Also, and this is unrelated to the actual question... Why don't these
> messages show up in the archive?
>
> http://apache-spark-user-list.1001560.n3.nabble.com/
>
> Ideally I'd want to post a link to our internal wiki for these questions,
> but can't find them in the archive.
>
> On 11 May 2017 at 07:16, lucas.gary@gmail.com <lu...@gmail.com> wrote:
>>
>> Looks like this isn't viable in spark 2.0.0 (and greater I presume).  I'm
>> pretty sure I came across this blog and ignored it due to that.
>>
>> Any other thoughts?  The linked tickets in:
>> https://issues.apache.org/jira/browse/SPARK-10063
>> https://issues.apache.org/jira/browse/HADOOP-13786
>> https://issues.apache.org/jira/browse/HADOOP-9565 look relevant too.
>>
>> On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com> wrote:
>>>
>>> Try using the DirectParquetOutputCommiter:
>>> http://dev.sortable.com/spark-directparquetoutputcommitter/
>>>
>>> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
>>> <lu...@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.
>>> >
>>> > 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'
>>> >
>>> > 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
>>
>>
>

---------------------------------------------------------------------
To unsubscribe e-mail: user-unsubscribe@spark.apache.org


Re: Spark <--> S3 flakiness

Posted by "lucas.gary@gmail.com" <lu...@gmail.com>.
Interesting, the links here: http://spark.apache.org/community.html
point to: http://apache-spark-user-list.1001560.n3.nabble.com/



On 11 May 2017 at 12:35, Vadim Semenov <va...@datadoghq.com> wrote:

> Use the official mailing list archive
>
> http://mail-archives.apache.org/mod_mbox/spark-user/201705.mbox/%
> 3cCAJYeQ0gh1FBHBAjb9GGHOgNhQOuogYdbA28Lnn262HfZzgFzyw@mail.gmail.com%3e
>
> On Thu, May 11, 2017 at 2:50 PM, lucas.gary@gmail.com <
> lucas.gary@gmail.com> wrote:
>
>> Also, and this is unrelated to the actual question... Why don't these
>> messages show up in the archive?
>>
>> http://apache-spark-user-list.1001560.n3.nabble.com/
>>
>> Ideally I'd want to post a link to our internal wiki for these questions,
>> but can't find them in the archive.
>>
>> On 11 May 2017 at 07:16, lucas.gary@gmail.com <lu...@gmail.com>
>> wrote:
>>
>>> Looks like this isn't viable in spark 2.0.0 (and greater I presume).
>>> I'm pretty sure I came across this blog and ignored it due to that.
>>>
>>> Any other thoughts?  The linked tickets in: https://issues.apache.org/
>>> jira/browse/SPARK-10063 https://issues.apache.org/jira/brows
>>> e/HADOOP-13786 https://issues.apache.org/jira/browse/HADOOP-9565 look
>>> relevant too.
>>>
>>> On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com> wrote:
>>>
>>>> Try using the DirectParquetOutputCommiter:
>>>> http://dev.sortable.com/spark-directparquetoutputcommitter/
>>>>
>>>> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
>>>> <lu...@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.
>>>> >
>>>> > 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(Abst
>>>> ractSessionInputBuffer.java:198)
>>>> >> at
>>>> >> org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>>>> tLengthInputStream.java:178)
>>>> >> at
>>>> >> org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>>>> tLengthInputStream.java:200)
>>>> >> at
>>>> >> org.apache.http.impl.io.ContentLengthInputStream.close(Conte
>>>> ntLengthInputStream.java:103)
>>>> >> at
>>>> >> org.apache.http.conn.BasicManagedEntity.streamClosed(BasicMa
>>>> nagedEntity.java:168)
>>>> >> at
>>>> >> org.apache.http.conn.EofSensorInputStream.checkClose(EofSens
>>>> orInputStream.java:228)
>>>> >> at
>>>> >> org.apache.http.conn.EofSensorInputStream.close(EofSensorInp
>>>> utStream.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'
>>>> >
>>>> > 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
>>>>
>>>
>>>
>>
>

Re: Spark <--> S3 flakiness

Posted by Vadim Semenov <va...@datadoghq.com>.
Use the official mailing list archive

http://mail-archives.apache.org/mod_mbox/spark-user/201705.mbox/%3cCAJYeQ0gh1FBHBAjb9GGHOgNhQOuogYdbA28Lnn262HfZzgFzyw@mail.gmail.com%3e

On Thu, May 11, 2017 at 2:50 PM, lucas.gary@gmail.com <lu...@gmail.com>
wrote:

> Also, and this is unrelated to the actual question... Why don't these
> messages show up in the archive?
>
> http://apache-spark-user-list.1001560.n3.nabble.com/
>
> Ideally I'd want to post a link to our internal wiki for these questions,
> but can't find them in the archive.
>
> On 11 May 2017 at 07:16, lucas.gary@gmail.com <lu...@gmail.com>
> wrote:
>
>> Looks like this isn't viable in spark 2.0.0 (and greater I presume).  I'm
>> pretty sure I came across this blog and ignored it due to that.
>>
>> Any other thoughts?  The linked tickets in: https://issues.apache.org/
>> jira/browse/SPARK-10063 https://issues.apache.org/jira/brows
>> e/HADOOP-13786 https://issues.apache.org/jira/browse/HADOOP-9565 look
>> relevant too.
>>
>> On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com> wrote:
>>
>>> Try using the DirectParquetOutputCommiter:
>>> http://dev.sortable.com/spark-directparquetoutputcommitter/
>>>
>>> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
>>> <lu...@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.
>>> >
>>> > 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(Abst
>>> ractSessionInputBuffer.java:198)
>>> >> at
>>> >> org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>>> tLengthInputStream.java:178)
>>> >> at
>>> >> org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>>> tLengthInputStream.java:200)
>>> >> at
>>> >> org.apache.http.impl.io.ContentLengthInputStream.close(Conte
>>> ntLengthInputStream.java:103)
>>> >> at
>>> >> org.apache.http.conn.BasicManagedEntity.streamClosed(BasicMa
>>> nagedEntity.java:168)
>>> >> at
>>> >> org.apache.http.conn.EofSensorInputStream.checkClose(EofSens
>>> orInputStream.java:228)
>>> >> at
>>> >> org.apache.http.conn.EofSensorInputStream.close(EofSensorInp
>>> utStream.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'
>>> >
>>> > 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
>>>
>>
>>
>

Re: Spark <--> S3 flakiness

Posted by "lucas.gary@gmail.com" <lu...@gmail.com>.
Also, and this is unrelated to the actual question... Why don't these
messages show up in the archive?

http://apache-spark-user-list.1001560.n3.nabble.com/

Ideally I'd want to post a link to our internal wiki for these questions,
but can't find them in the archive.

On 11 May 2017 at 07:16, lucas.gary@gmail.com <lu...@gmail.com> wrote:

> Looks like this isn't viable in spark 2.0.0 (and greater I presume).  I'm
> pretty sure I came across this blog and ignored it due to that.
>
> Any other thoughts?  The linked tickets in: https://issues.apache.org/
> jira/browse/SPARK-10063 https://issues.apache.org/jira/browse/HADOOP-13786
>  https://issues.apache.org/jira/browse/HADOOP-9565 look relevant too.
>
> On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com> wrote:
>
>> Try using the DirectParquetOutputCommiter:
>> http://dev.sortable.com/spark-directparquetoutputcommitter/
>>
>> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
>> <lu...@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.
>> >
>> > 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(Abst
>> ractSessionInputBuffer.java:198)
>> >> at
>> >> org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>> tLengthInputStream.java:178)
>> >> at
>> >> org.apache.http.impl.io.ContentLengthInputStream.read(Conten
>> tLengthInputStream.java:200)
>> >> at
>> >> org.apache.http.impl.io.ContentLengthInputStream.close(Conte
>> ntLengthInputStream.java:103)
>> >> at
>> >> org.apache.http.conn.BasicManagedEntity.streamClosed(BasicMa
>> nagedEntity.java:168)
>> >> at
>> >> org.apache.http.conn.EofSensorInputStream.checkClose(EofSens
>> orInputStream.java:228)
>> >> at
>> >> org.apache.http.conn.EofSensorInputStream.close(EofSensorInp
>> utStream.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'
>> >
>> > 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
>>
>
>

Re: Spark <--> S3 flakiness

Posted by "lucas.gary@gmail.com" <lu...@gmail.com>.
Looks like this isn't viable in spark 2.0.0 (and greater I presume).  I'm
pretty sure I came across this blog and ignored it due to that.

Any other thoughts?  The linked tickets in:
https://issues.apache.org/jira/browse/SPARK-10063
https://issues.apache.org/jira/browse/HADOOP-13786
https://issues.apache.org/jira/browse/HADOOP-9565 look relevant too.

On 10 May 2017 at 22:24, Miguel Morales <th...@gmail.com> wrote:

> Try using the DirectParquetOutputCommiter:
> http://dev.sortable.com/spark-directparquetoutputcommitter/
>
> On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
> <lu...@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.
> >
> > 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'
> >
> > 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
>

Re: Spark <--> S3 flakiness

Posted by Miguel Morales <th...@gmail.com>.
Try using the DirectParquetOutputCommiter:
http://dev.sortable.com/spark-directparquetoutputcommitter/

On Wed, May 10, 2017 at 10:07 PM, lucas.gary@gmail.com
<lu...@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.
>
> 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'
>
> 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

---------------------------------------------------------------------
To unsubscribe e-mail: user-unsubscribe@spark.apache.org