Re: Spark, S3A, and 503 SlowDown / rate limit issues

2017-07-12 Thread Steve Loughran

On 10 Jul 2017, at 21:57, Everett Anderson 
> wrote:

Hey,

Thanks for the responses, guys!

On Thu, Jul 6, 2017 at 7:08 AM, Steve Loughran 
> wrote:

On 5 Jul 2017, at 14:40, Vadim Semenov 
> wrote:

Are you sure that you use S3A?
Because EMR says that they do not support S3A

https://aws.amazon.com/premiumsupport/knowledge-center/emr-file-system-s3/
> Amazon EMR does not currently support use of the Apache Hadoop S3A file 
> system.

Oof. I figured they didn't offer technical support for S3A, but didn't know 
that there was something saying EMR does not support use of S3A. My impression 
was that many people were using it and it's the recommended S3 library in 
Hadoop 2.7+ from Hadoop's point of 
view.

We're using it rather than S3N because we use encrypted buckets, and I don't 
think S3N supports picking up credentials from a machine role. Also, it was a 
bit distressing that it's unmaintained and has open bugs.

We're S3A rather than EMRFS because we have a setup where we submit work to a 
cluster via spark-submit run outside the cluster master node with --master 
yarn. When you do this, the Hadoop configuration accessible to spark-submit 
overrides that of the EMR cluster itself. If you use a configuration that uses 
EMRFS and any of the resources (like the JAR) you give to spark-submit are on 
S3, spark-submit will instantiate the EMRFS FileSystem impl, which is currently 
only available on the cluster, and fail. That said, we could work around this 
by resetting the configuration in code.


or, if you are using the URL s3:// to refer to amazon EMRs, just edit your app 
config so that fs.s3.impl=org.apache.hadoop.fs.s3a.S3AFileSystem  and use s3:// 
everywhere (use the fs.s3a. prefix for configuring s3 though)



I think that the HEAD requests come from the `createBucketIfNotExists` in the 
AWS S3 library that checks if the bucket exists every time you do a PUT 
request, i.e. creates a HEAD request.

You can disable that by setting `fs.s3.buckets.create.enabled` to `false`
http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-plan-upload-s3.html
Oh, interesting. We are definitely seeing a ton of HEAD requests, which might 
be that. It looks like the `fs.s3.buckets.create.enabled` is an EMRFS option, 
though, not one common to the Hadoop S3 FileSystem implementations. Does that 
sound right?




Yeah, I'd like to see the stack traces before blaming S3A and the ASF codebase

(Sorry, to be clear -- I'm not trying to blame S3A. I figured someone else 
might've hit this and bet we had just misconfigured something or were doing 
this the wrong way.)

no worries,..if you are seeing problems, it's important to know where they are 
surfacing.



One thing I do know is that the shipping S3A client doesn't have any explicit 
handling of 503/retry events. I know that: 
https://issues.apache.org/jira/browse/HADOOP-14531

There is some retry logic in bits of the AWS SDK related to file upload: that 
may log and retry, but in all the operations listing files, getting their 
details, etc: no resilience to throttling.

If it is surfacing against s3a, there isn't anything which can immediately be 
done to fix it, other than "spread your data around more buckets". Do attach 
the stack trace you get under 
https://issues.apache.org/jira/browse/HADOOP-14381 though: I'm about half-way 
through the resilience code (& fault injection needed to test it). The more 
where I can see problems arise, the more confident I can be that those 
codepaths will be resilient.

Will do!

We did end up finding that some of our jobs were sharding data way too finely, 
ending up with 5-10k+ tiny Parquet shards per table. This happened when we 
unioned many Spark DataFrames together without doing a repartition or coalesce 
afterwards. After throwing in a repartition (to additionally balance the output 
shards) we haven't seen the error, again, but our graphs of S3 HEAD requests 
are still rather alarmingly high.



treewalking can be expensive that way; the more dirs you have, the more things 
look around.

If you are using S3A, and Hadoop 2.8+, log the toString() value of the FS after 
your submission. It'll give you a list of all the stats it collects, including 
details fo high level API calls alongside low level HTTP requests: 
https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java






Re: Spark, S3A, and 503 SlowDown / rate limit issues

2017-07-10 Thread Everett Anderson
Hey,

Thanks for the responses, guys!

On Thu, Jul 6, 2017 at 7:08 AM, Steve Loughran 
wrote:

>
> On 5 Jul 2017, at 14:40, Vadim Semenov 
> wrote:
>
> Are you sure that you use S3A?
> Because EMR says that they do not support S3A
>
> https://aws.amazon.com/premiumsupport/knowledge-center/emr-file-system-s3/
> > Amazon EMR does not currently support use of the Apache Hadoop S3A file
> system.
>
>
Oof. I figured they didn't offer technical support for S3A, but didn't know
that there was something saying EMR does not support use of S3A. My
impression was that many people were using it and it's the recommended S3
library in Hadoop 2.7+  from
Hadoop's point of view.

We're using it rather than S3N because we use encrypted buckets, and I
don't think S3N supports picking up credentials from a machine role. Also,
it was a bit distressing that it's unmaintained and has open bugs.

We're S3A rather than EMRFS because we have a setup where we submit work to
a cluster via spark-submit run outside the cluster master node with
--master yarn. When you do this, the Hadoop configuration accessible to
spark-submit overrides that of the EMR cluster itself. If you use a
configuration that uses EMRFS and any of the resources (like the JAR) you
give to spark-submit are on S3, spark-submit will instantiate the EMRFS
FileSystem impl, which is currently only available on the cluster, and
fail. That said, we could work around this by resetting the configuration
in code.


>
> I think that the HEAD requests come from the `createBucketIfNotExists` in
> the AWS S3 library that checks if the bucket exists every time you do a PUT
> request, i.e. creates a HEAD request.
>
> You can disable that by setting `fs.s3.buckets.create.enabled` to `false`
> http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-
> plan-upload-s3.html
>
> Oh, interesting. We are definitely seeing a ton of HEAD requests, which
might be that. It looks like the `fs.s3.buckets.create.enabled` is an EMRFS
option, though, not one common to the Hadoop S3 FileSystem implementations.
Does that sound right?

>
>
>
>
> Yeah, I'd like to see the stack traces before blaming S3A and the ASF
> codebase
>

(Sorry, to be clear -- I'm not trying to blame S3A. I figured someone else
might've hit this and bet we had just misconfigured something or were doing
this the wrong way.)


>
> One thing I do know is that the shipping S3A client doesn't have any
> explicit handling of 503/retry events. I know that:
> https://issues.apache.org/jira/browse/HADOOP-14531
>
> There is some retry logic in bits of the AWS SDK related to file upload:
> that may log and retry, but in all the operations listing files, getting
> their details, etc: no resilience to throttling.
>
> If it is surfacing against s3a, there isn't anything which can immediately
> be done to fix it, other than "spread your data around more buckets". Do
> attach the stack trace you get under https://issues.apache.or
> g/jira/browse/HADOOP-14381 though: I'm about half-way through the
> resilience code (& fault injection needed to test it). The more where I can
> see problems arise, the more confident I can be that those codepaths will
> be resilient.
>

Will do!

We did end up finding that some of our jobs were sharding data way too
finely, ending up with 5-10k+ tiny Parquet shards per table. This happened
when we unioned many Spark DataFrames together without doing a repartition
or coalesce afterwards. After throwing in a repartition (to additionally
balance the output shards) we haven't seen the error, again, but our graphs
of S3 HEAD requests are still rather alarmingly high.




>
>
> On Thu, Jun 29, 2017 at 4:56 PM, Everett Anderson <
> ever...@nuna.com.invalid> wrote:
>
>> Hi,
>>
>> We're using Spark 2.0.2 + Hadoop 2.7.3 on AWS EMR with S3A for direct I/O
>> from/to S3 from our Spark jobs. We set 
>> mapreduce.fileoutputcommitter.algorithm.version=2
>> and are using encrypted S3 buckets.
>>
>> This has been working fine for us, but perhaps as we've been running more
>> jobs in parallel, we've started getting errors like
>>
>> Status Code: 503, AWS Service: Amazon S3, AWS Request ID: ..., AWS Error
>> Code: SlowDown, AWS Error Message: Please reduce your request rate., S3
>> Extended Request ID: ...
>>
>> We enabled CloudWatch S3 request metrics for one of our buckets and I was
>> a little alarmed to see spikes of over 800k S3 requests over a minute or
>> so, with the bulk of them HEAD requests.
>>
>> We read and write Parquet files, and most tables have around 50
>> shards/parts, though some have up to 200. I imagine there's additional
>> parallelism when reading a shard in Parquet, though.
>>
>> Has anyone else encountered this? How did you solve it?
>>
>> I'd sure prefer to avoid copying all our data in and out of HDFS for each
>> job, if possible.
>>
>> Thanks!
>>
>>
>
>


Re: Spark, S3A, and 503 SlowDown / rate limit issues

2017-07-06 Thread Steve Loughran

On 5 Jul 2017, at 14:40, Vadim Semenov 
> wrote:

Are you sure that you use S3A?
Because EMR says that they do not support S3A

https://aws.amazon.com/premiumsupport/knowledge-center/emr-file-system-s3/
> Amazon EMR does not currently support use of the Apache Hadoop S3A file 
> system.

I think that the HEAD requests come from the `createBucketIfNotExists` in the 
AWS S3 library that checks if the bucket exists every time you do a PUT 
request, i.e. creates a HEAD request.

You can disable that by setting `fs.s3.buckets.create.enabled` to `false`
http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-plan-upload-s3.html



Yeah, I'd like to see the stack traces before blaming S3A and the ASF codebase

One thing I do know is that the shipping S3A client doesn't have any explicit 
handling of 503/retry events. I know that: 
https://issues.apache.org/jira/browse/HADOOP-14531

There is some retry logic in bits of the AWS SDK related to file upload: that 
may log and retry, but in all the operations listing files, getting their 
details, etc: no resilience to throttling.

If it is surfacing against s3a, there isn't anything which can immediately be 
done to fix it, other than "spread your data around more buckets". Do attach 
the stack trace you get under 
https://issues.apache.org/jira/browse/HADOOP-14381 though: I'm about half-way 
through the resilience code (& fault injection needed to test it). The more 
where I can see problems arise, the more confident I can be that those 
codepaths will be resilient.


On Thu, Jun 29, 2017 at 4:56 PM, Everett Anderson 
> wrote:
Hi,

We're using Spark 2.0.2 + Hadoop 2.7.3 on AWS EMR with S3A for direct I/O 
from/to S3 from our Spark jobs. We set 
mapreduce.fileoutputcommitter.algorithm.version=2 and are using encrypted S3 
buckets.

This has been working fine for us, but perhaps as we've been running more jobs 
in parallel, we've started getting errors like

Status Code: 503, AWS Service: Amazon S3, AWS Request ID: ..., AWS Error Code: 
SlowDown, AWS Error Message: Please reduce your request rate., S3 Extended 
Request ID: ...

We enabled CloudWatch S3 request metrics for one of our buckets and I was a 
little alarmed to see spikes of over 800k S3 requests over a minute or so, with 
the bulk of them HEAD requests.

We read and write Parquet files, and most tables have around 50 shards/parts, 
though some have up to 200. I imagine there's additional parallelism when 
reading a shard in Parquet, though.

Has anyone else encountered this? How did you solve it?

I'd sure prefer to avoid copying all our data in and out of HDFS for each job, 
if possible.

Thanks!





Re: Spark, S3A, and 503 SlowDown / rate limit issues

2017-07-05 Thread Vadim Semenov
Are you sure that you use S3A?
Because EMR says that they do not support S3A

https://aws.amazon.com/premiumsupport/knowledge-center/emr-file-system-s3/
> Amazon EMR does not currently support use of the Apache Hadoop S3A file
system.

I think that the HEAD requests come from the `createBucketIfNotExists` in
the AWS S3 library that checks if the bucket exists every time you do a PUT
request, i.e. creates a HEAD request.

You can disable that by setting `fs.s3.buckets.create.enabled` to `false`
http://docs.aws.amazon.com/emr/latest/ManagementGuide/emr-plan-upload-s3.html

On Thu, Jun 29, 2017 at 4:56 PM, Everett Anderson 
wrote:

> Hi,
>
> We're using Spark 2.0.2 + Hadoop 2.7.3 on AWS EMR with S3A for direct I/O
> from/to S3 from our Spark jobs. We set mapreduce.
> fileoutputcommitter.algorithm.version=2 and are using encrypted S3
> buckets.
>
> This has been working fine for us, but perhaps as we've been running more
> jobs in parallel, we've started getting errors like
>
> Status Code: 503, AWS Service: Amazon S3, AWS Request ID: ..., AWS Error
> Code: SlowDown, AWS Error Message: Please reduce your request rate., S3
> Extended Request ID: ...
>
> We enabled CloudWatch S3 request metrics for one of our buckets and I was
> a little alarmed to see spikes of over 800k S3 requests over a minute or
> so, with the bulk of them HEAD requests.
>
> We read and write Parquet files, and most tables have around 50
> shards/parts, though some have up to 200. I imagine there's additional
> parallelism when reading a shard in Parquet, though.
>
> Has anyone else encountered this? How did you solve it?
>
> I'd sure prefer to avoid copying all our data in and out of HDFS for each
> job, if possible.
>
> Thanks!
>
>