HADOOP-15076. Enhance S3A troubleshooting documents and add a performance document.

Contributed by Steve Loughran.

(cherry picked from commit c761e658f6594c4e519ed39ef36669de2c5cee15)
This commit is contained in:
Steve Loughran 2018-02-15 14:57:56 +00:00
parent c9a373fb14
commit b27ab7dd81
4 changed files with 1029 additions and 340 deletions

View File

@ -37,6 +37,8 @@ and keys with which the file was encrypted.
* You can use AWS bucket policies to mandate encryption rules for a bucket.
* You can use S3A per-bucket configuration to ensure that S3A clients use encryption
policies consistent with the mandated rules.
* You can use S3 Default Encryption to encrypt data without needing to
set anything in the client.
* Changing the encryption options on the client does not change how existing
files were encrypted, except when the files are renamed.
* For all mechanisms other than SSE-C, clients do not need any configuration
@ -58,9 +60,10 @@ The server-side "SSE" encryption is performed with symmetric AES256 encryption;
S3 offers different mechanisms for actually defining the key to use.
There are thrre key management mechanisms, which in order of simplicity of use,
There are four key management mechanisms, which in order of simplicity of use,
are:
* S3 Default Encryption
* SSE-S3: an AES256 key is generated in S3, and saved alongside the data.
* SSE-KMS: an AES256 key is generated in S3, and encrypted with a secret key provided
by Amazon's Key Management Service, a key referenced by name in the uploading client.
@ -68,6 +71,19 @@ by Amazon's Key Management Service, a key referenced by name in the uploading cl
to encrypt and decrypt the data.
## <a name="sse-s3"></a> S3 Default Encryption
This feature allows the administrators of the AWS account to set the "default"
encryption policy on a bucket -the encryption to use if the client does
not explicitly declare an encryption algorithm.
[S3 Default Encryption for S3 Buckets](https://docs.aws.amazon.com/AmazonS3/latest/dev/bucket-encryption.html)
This supports SSE-S3 and SSE-KMS.
There is no need to set anything up in the client: do it in the AWS console.
## <a name="sse-s3"></a> SSE-S3 Amazon S3-Managed Encryption Keys
In SSE-S3, all keys and secrets are managed inside S3. This is the simplest encryption mechanism.
@ -413,7 +429,6 @@ How can you do that from Hadoop? With `rename()`.
The S3A client mimics a real filesystem's' rename operation by copying all the
source files to the destination paths, then deleting the old ones.
If you do a rename()
Note: this does not work for SSE-C, because you cannot set a different key
for reading as for writing, and you must supply that key for reading. There
@ -421,7 +436,7 @@ you need to copy one bucket to a different bucket, one with a different key.
Use `distCp`for this, with per-bucket encryption policies.
## <a name="Troubleshooting"></a> Troubleshooting Encryption
## <a name="troubleshooting"></a> Troubleshooting Encryption
The [troubleshooting](./troubleshooting_s3a.html) document covers
stack traces which may surface when working with encrypted data.

View File

@ -25,6 +25,7 @@ Please use `s3a:` as the connector to data hosted in S3 with Apache Hadoop.**
See also:
* [Encryption](./encryption.html)
* [Performance](./performance.html)
* [S3Guard](./s3guard.html)
* [Troubleshooting](./troubleshooting_s3a.html)
* [Committing work to S3 with the "S3A Committers"](./committers.html)
@ -1580,80 +1581,8 @@ The S3A Filesystem client supports the notion of input policies, similar
to that of the Posix `fadvise()` API call. This tunes the behavior of the S3A
client to optimise HTTP GET requests for the different use cases.
*"sequential"*
Read through the file, possibly with some short forward seeks.
The whole document is requested in a single HTTP request; forward seeks
within the readahead range are supported by skipping over the intermediate
data.
This is leads to maximum read throughput —but with very expensive
backward seeks.
*"normal" (default)*
The "Normal" policy starts off reading a file in "sequential" mode,
but if the caller seeks backwards in the stream, it switches from
sequential to "random".
This policy effectively recognizes the initial read pattern of columnar
storage formats (e.g. Apache ORC and Apache Parquet), which seek to the end
of a file, read in index data and then seek backwards to selectively read
columns. The first seeks may be be expensive compared to the random policy,
however the overall process is much less expensive than either sequentially
reading through a file with the "random" policy, or reading columnar data
with the "sequential" policy. When the exact format/recommended
seek policy of data are known in advance, this policy
*"random"*
Optimised for random IO, specifically the Hadoop `PositionedReadable`
operations —though `seek(offset); read(byte_buffer)` also benefits.
Rather than ask for the whole file, the range of the HTTP request is
set to that that of the length of data desired in the `read` operation
(Rounded up to the readahead value set in `setReadahead()` if necessary).
By reducing the cost of closing existing HTTP requests, this is
highly efficient for file IO accessing a binary file
through a series of `PositionedReadable.read()` and `PositionedReadable.readFully()`
calls. Sequential reading of a file is expensive, as now many HTTP requests must
be made to read through the file.
For operations simply reading through a file: copying, distCp, reading
Gzipped or other compressed formats, parsing .csv files, etc, the `sequential`
policy is appropriate. This is the default: S3A does not need to be configured.
For the specific case of high-performance random access IO, the `random` policy
may be considered. The requirements are:
* Data is read using the `PositionedReadable` API.
* Long distance (many MB) forward seeks
* Backward seeks as likely as forward seeks.
* Little or no use of single character `read()` calls or small `read(buffer)`
calls.
* Applications running close to the S3 data store. That is: in EC2 VMs in
the same datacenter as the S3 instance.
The desired fadvise policy must be set in the configuration option
`fs.s3a.experimental.input.fadvise` when the filesystem instance is created.
That is: it can only be set on a per-filesystem basis, not on a per-file-read
basis.
<property>
<name>fs.s3a.experimental.input.fadvise</name>
<value>random</value>
<description>Policy for reading files.
Values: 'random', 'sequential' or 'normal'
</description>
</property>
[HDFS-2744](https://issues.apache.org/jira/browse/HDFS-2744),
*Extend FSDataInputStream to allow fadvise* proposes adding a public API
to set fadvise policies on input streams. Once implemented,
this will become the supported mechanism used for configuring the input IO policy.
See [Improving data input performance through fadvise](./performance.html#fadvise)
for the details.
##<a name="metrics"></a>Metrics

View File

@ -0,0 +1,518 @@
<!---
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License. See accompanying LICENSE file.
-->
# Maximizing Performance when working with the S3A Connector
<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
## <a name="introduction"></a> Introduction
S3 is slower to work with than HDFS, even on virtual clusters running on
Amazon EC2.
That's because its a very different system, as you can see:
| Feature | HDFS | S3 through the S3A connector |
|---------|------|------------------------------|
| communication | RPC | HTTP GET/PUT/HEAD/LIST/COPY requests |
| data locality | local storage | remote S3 servers |
| replication | multiple datanodes | asynchronous after upload |
| consistency | consistent data and listings | eventual consistent for listings, deletes and updates |
| bandwidth | best: local IO, worst: datacenter network | bandwidth between servers and S3 |
| latency | low | high, especially for "low cost" directory operations |
| rename | fast, atomic | slow faked rename through COPY & DELETE|
| delete | fast, atomic | fast for a file, slow & non-atomic for directories |
| writing| incremental | in blocks; not visible until the writer is closed |
| reading | seek() is fast | seek() is slow and expensive |
| IOPs | limited only by hardware | callers are throttled to shards in an s3 bucket |
| Security | Posix user+group; ACLs | AWS Roles and policies |
From a performance perspective, key points to remember are:
* S3 throttles bucket access across all callers: adding workers can make things worse.
* EC2 VMs have network IO throttled based on the VM type.
* Directory rename and copy operations take *much* longer the more objects and data there is.
The slow performance of `rename()` surfaces during the commit phase of jobs,
applications like `DistCP`, and elsewhere.
* seek() calls when reading a file can force new HTTP requests.
This can make reading columnar Parquet/ORC data expensive.
Overall, although the S3A connector makes S3 look like a file system,
it isn't, and some attempts to preserve the metaphor are "aggressively suboptimal".
To make most efficient use of S3, care is needed.
## <a name="s3guard"></a> Speeding up directory listing operations through S3Guard
[S3Guard](s3guard.html) provides significant speedups for operations which
list files a lot. This includes the setup of all queries against data:
MapReduce, Hive and Spark, as well as DistCP.
Experiment with using it to see what speedup it delivers.
## <a name="fadvise"></a> Improving data input performance through fadvise
The S3A Filesystem client supports the notion of input policies, similar
to that of the Posix `fadvise()` API call. This tunes the behavior of the S3A
client to optimise HTTP GET requests for the different use cases.
### fadvise `sequential`
Read through the file, possibly with some short forward seeks.
The whole document is requested in a single HTTP request; forward seeks
within the readahead range are supported by skipping over the intermediate
data.
This delivers maximum sequential throughput —but with very expensive
backward seeks.
Applications reading a file in bulk (DistCP, any copy operations) should use
sequential access, as should those reading data from gzipped `.gz` files.
Because the "normal" fadvise policy starts off in sequential IO mode,
there is rarely any need to explicit request this policy.
### fadvise `random`
Optimised for random IO, specifically the Hadoop `PositionedReadable`
operations —though `seek(offset); read(byte_buffer)` also benefits.
Rather than ask for the whole file, the range of the HTTP request is
set to that that of the length of data desired in the `read` operation
(Rounded up to the readahead value set in `setReadahead()` if necessary).
By reducing the cost of closing existing HTTP requests, this is
highly efficient for file IO accessing a binary file
through a series of `PositionedReadable.read()` and `PositionedReadable.readFully()`
calls. Sequential reading of a file is expensive, as now many HTTP requests must
be made to read through the file: there's a delay between each GET operation.
Random IO is best for IO with seek-heavy characteristics:
* Data is read using the `PositionedReadable` API.
* Long distance (many MB) forward seeks
* Backward seeks as likely as forward seeks.
* Little or no use of single character `read()` calls or small `read(buffer)`
calls.
* Applications running close to the S3 data store. That is: in EC2 VMs in
the same datacenter as the S3 instance.
The desired fadvise policy must be set in the configuration option
`fs.s3a.experimental.input.fadvise` when the filesystem instance is created.
That is: it can only be set on a per-filesystem basis, not on a per-file-read
basis.
```xml
<property>
<name>fs.s3a.experimental.input.fadvise</name>
<value>random</value>
<description>
Policy for reading files.
Values: 'random', 'sequential' or 'normal'
</description>
</property>
```
[HDFS-2744](https://issues.apache.org/jira/browse/HDFS-2744),
*Extend FSDataInputStream to allow fadvise* proposes adding a public API
to set fadvise policies on input streams. Once implemented,
this will become the supported mechanism used for configuring the input IO policy.
### fadvise `normal` (default)
The `normal` policy starts off reading a file in `sequential` mode,
but if the caller seeks backwards in the stream, it switches from
sequential to `random`.
This policy essentially recognizes the initial read pattern of columnar
storage formats (e.g. Apache ORC and Apache Parquet), which seek to the end
of a file, read in index data and then seek backwards to selectively read
columns. The first seeks may be be expensive compared to the random policy,
however the overall process is much less expensive than either sequentially
reading through a file with the `random` policy, or reading columnar data
with the `sequential` policy.
## <a name="commit"></a> Committing Work in MapReduce and Spark
Hadoop MapReduce, Apache Hive and Apache Spark all write their work
to HDFS and similar filesystems.
When using S3 as a destination, this is slow because of the way `rename()`
is mimicked with copy and delete.
If committing output takes a long time, it is because you are using the standard
`FileOutputCommitter`. If you are doing this on any S3 endpoint which lacks
list consistency (Amazon S3 without [S3Guard](s3guard.html)), this committer
is at risk of losing data!
*Your problem may appear to be performance, but that is a symptom
of the underlying problem: the way S3A fakes rename operations means that
the rename cannot be safely be used in output-commit algorithms.*
Fix: Use one of the dedicated [S3A Committers](committers.md).
## <a name="tuning"></a> Options to Tune
### <a name="pooling"></a> Thread and connection pool sizes.
Each S3A client interacting with a single bucket, as a single user, has its
own dedicated pool of open HTTP 1.1 connections alongside a pool of threads used
for upload and copy operations.
The default pool sizes are intended to strike a balance between performance
and memory/thread use.
You can have a larger pool of (reused) HTTP connections and threads
for parallel IO (especially uploads) by setting the properties
| property | meaning | default |
|----------|---------|---------|
| `fs.s3a.threads.max`| Threads in the AWS transfer manager| 10 |
| `fs.s3a.connection.maximum`| Maximum number of HTTP connections | 10|
We recommend using larger values for processes which perform
a lot of IO: `DistCp`, Spark Workers and similar.
```xml
<property>
<name>fs.s3a.threads.max</name>
<value>20</value>
</property>
<property>
<name>fs.s3a.connection.maximum</name>
<value>20</value>
</property>
```
Be aware, however, that processes which perform many parallel queries
may consume large amounts of resources if each query is working with
a different set of s3 buckets, or are acting on behalf of different users.
### For large data uploads, tune the block size: `fs.s3a.block.size`
When uploading data, it is uploaded in blocks set by the option
`fs.s3a.block.size`; default value "32M" for 32 Megabytes.
If a larger value is used, then more data is buffered before the upload
begins:
```xml
<property>
<name>fs.s3a.block.size</name>
<value>128M</value>
</property>
```
This means that fewer PUT/POST requests are made of S3 to upload data,
which reduces the likelihood that S3 will throttle the client(s)
### Maybe: Buffer Write Data in Memory
When large files are being uploaded, blocks are saved to disk and then
queued for uploading, with multiple threads uploading different blocks
in parallel.
The blocks can be buffered in memory by setting the option
`fs.s3a.fast.upload.buffer` to `bytebuffer`, or, for on-heap storage
`array`.
1. Switching to in memory-IO reduces disk IO, and can be faster if the bandwidth
to the S3 store is so high that the disk IO becomes the bottleneck.
This can have a tangible benefit when working with on-premise S3-compatible
object stores with very high bandwidth to servers.
It is very easy to run out of memory when buffering to it; the option
`fs.s3a.fast.upload.active.blocks"` exists to tune how many active blocks
a single output stream writing to S3 may have queued at a time.
As the size of each buffered block is determined by the value of `fs.s3a.block.size`,
the larger the block size, the more likely you will run out of memory.
## <a name="distcp"></a> DistCP
DistCP can be slow, especially if the parameters and options for the operation
are not tuned for working with S3.
To exacerbate the issue, DistCP invariably puts heavy load against the
bucket being worked with, which will cause S3 to throttle requests.
It will throttle: directory operations, uploads of new data, and delete operations,
amongst other things
### DistCP: Options to Tune
* `-numListstatusThreads <threads>` : set to something higher than the default (1).
* `-bandwidth <mb>` : use to limit the upload bandwidth per worker
* `-m <maps>` : limit the number of mappers, hence the load on the S3 bucket.
Adding more maps with the `-m` option does not guarantee better performance;
it may just increase the amount of throttling which takes place.
A smaller number of maps with a higher bandwidth per map can be more efficient.
### DistCP: Options to Avoid.
DistCp's `-atomic` option copies up data into a directory, then renames
it into place, which is the where the copy takes place. This is a performance
killer.
* Do not use the `-atomic` option.
* The `-append` operation is not supported on S3; avoid.
* `-p` S3 does not have a POSIX-style permission model; this will fail.
### DistCP: Parameters to Tune
1. As discussed [earlier](#pooling), use large values for
`fs.s3a.threads.max` and `fs.s3a.connection.maximum`.
1. Make sure that the bucket is using `sequential` or `normal` fadvise seek policies,
that is, `fs.s3a.experimental.fadvise` is not set to `random`
1. Perform listings in parallel by setting `-numListstatusThreads`
to a higher number. Make sure that `fs.s3a.connection.maximum`
is equal to or greater than the value used.
1. If using `-delete`, set `fs.trash.interval` to 0 to avoid the deleted
objects from being copied to a trash directory.
*DO NOT* switch `fs.s3a.fast.upload.buffer` to buffer in memory.
If one distcp mapper runs out of memory it will fail,
and that runs the risk of failing the entire job.
It is safer to keep the default value, `disk`.
What is potentially useful is uploading in bigger blocks; this is more
efficient in terms of HTTP connection use, and reduce the IOP rate against
the S3 bucket/shard.
```xml
<property>
<name>fs.s3a.threads.max</name>
<value>20</value>
</property>
<property>
<name>fs.s3a.connection.maximum</name>
<value>30</value>
<descriptiom>
Make greater than both fs.s3a.threads.max and -numListstatusThreads
</descriptiom>
</property>
<property>
<name>fs.s3a.experimental.fadvise</name>
<value>normal</value>
</property>
<property>
<name>fs.s3a.block.size</name>
<value>128M</value>
</property>
<property>
<name>fs.s3a.fast.upload.buffer</name>
<value>disk</value>
</property>
<property>
<name>fs.trash.interval</name>
<value>0</value>
</property>
```
## <a name="rm"></a> hadoop shell commands `fs -rm`
The `hadoop fs -rm` command can rename the file under `.Trash` rather than
deleting it. Use `-skipTrash` to eliminate that step.
This can be set in the property `fs.trash.interval`; while the default is 0,
most HDFS deployments have it set to a non-zero value to reduce the risk of
data loss.
```xml
<property>
<name>fs.trash.interval</name>
<value>0</value>
</property>
```
## <a name="load balancing"></a> Improving S3 load-balancing behavior
Amazon S3 uses a set of front-end servers to provide access to the underlying data.
The choice of which front-end server to use is handled via load-balancing DNS
service: when the IP address of an S3 bucket is looked up, the choice of which
IP address to return to the client is made based on the the current load
of the front-end servers.
Over time, the load across the front-end changes, so those servers considered
"lightly loaded" will change. If the DNS value is cached for any length of time,
your application may end up talking to an overloaded server. Or, in the case
of failures, trying to talk to a server that is no longer there.
And by default, for historical security reasons in the era of applets,
the DNS TTL of a JVM is "infinity".
To work with AWS better, set the DNS time-to-live of an application which
works with S3 to something lower.
See [AWS documentation](http://docs.aws.amazon.com/AWSSdkDocsJava/latest/DeveloperGuide/java-dg-jvm-ttl.html).
## <a name="network_performance"></a> Troubleshooting network performance
An example of this is covered in [HADOOP-13871](https://issues.apache.org/jira/browse/HADOOP-13871).
1. For public data, use `curl`:
curl -O https://landsat-pds.s3.amazonaws.com/scene_list.gz
1. Use `nettop` to monitor a processes connections.
## <a name="throttling"></a> Throttling
When many requests are made of a specific S3 bucket (or shard inside it),
S3 will respond with a 503 "throttled" response.
Throttling can be recovered from, provided overall load decreases.
Furthermore, because it is sent before any changes are made to the object store,
is inherently idempotent. For this reason, the client will always attempt to
retry throttled requests.
The limit of the number of times a throttled request can be retried,
and the exponential interval increase between attempts, can be configured
independently of the other retry limits.
```xml
<property>
<name>fs.s3a.retry.throttle.limit</name>
<value>20</value>
<description>
Number of times to retry any throttled request.
</description>
</property>
<property>
<name>fs.s3a.retry.throttle.interval</name>
<value>500ms</value>
<description>
Interval between retry attempts on throttled requests.
</description>
</property>
```
If a client is failing due to `AWSServiceThrottledException` failures,
increasing the interval and limit *may* address this. However, it
it is a sign of AWS services being overloaded by the sheer number of clients
and rate of requests. Spreading data across different buckets, and/or using
a more balanced directory structure may be beneficial.
Consult [the AWS documentation](http://docs.aws.amazon.com/AmazonS3/latest/dev/request-rate-perf-considerations.html).
Reading or writing data encrypted with SSE-KMS forces S3 to make calls of
the AWS KMS Key Management Service, which comes with its own
[Request Rate Limits](http://docs.aws.amazon.com/kms/latest/developerguide/limits.html).
These default to 1200/second for an account, across all keys and all uses of
them, which, for S3 means: across all buckets with data encrypted with SSE-KMS.
### <a name="minimizing_throttling"></a> Tips to Keep Throttling down
If you are seeing a lot of throttling responses on a large scale
operation like a `distcp` copy, *reduce* the number of processes trying
to work with the bucket (for distcp: reduce the number of mappers with the
`-m` option).
If you are reading or writing lists of files, if you can randomize
the list so they are not processed in a simple sorted order, you may
reduce load on a specific shard of S3 data, so potentially increase throughput.
An S3 Bucket is throttled by requests coming from all
simultaneous clients. Different applications and jobs may interfere with
each other: consider that when troubleshooting.
Partitioning data into different buckets may help isolate load here.
If you are using data encrypted with SSE-KMS, then the
will also apply: these are stricter than the S3 numbers.
If you believe that you are reaching these limits, you may be able to
get them increased.
Consult [the KMS Rate Limit documentation](http://docs.aws.amazon.com/kms/latest/developerguide/limits.html).
### <a name="s3guard_throttling"></a> S3Guard and Throttling
S3Guard uses DynamoDB for directory and file lookups;
it is rate limited to the amount of (guaranteed) IO purchased for a
table.
To see the allocated capacity of a bucket, the `hadoop s3guard bucket-info s3a://bucket`
command will print out the allocated capacity.
If significant throttling events/rate is observed here, the pre-allocated
IOPs can be increased with the `hadoop s3guard set-capacity` command, or
through the AWS Console. Throttling events in S3Guard are noted in logs, and
also in the S3A metrics `s3guard_metadatastore_throttle_rate` and
`s3guard_metadatastore_throttled`.
If you are using DistCP for a large backup to/from a S3Guarded bucket, it is
actually possible to increase the capacity for the duration of the operation.
## <a name="coding"></a> Best Practises for Code
Here are some best practises if you are writing applications to work with
S3 or any other object store through the Hadoop APIs.
Use `listFiles(path, recursive)` over `listStatus(path)`.
The recursive `listFiles()` call can enumerate all dependents of a path
in a single LIST call, irrespective of how deep the path is.
In contrast, any directory tree-walk implemented in the client is issuing
multiple HTTP requests to scan each directory, all the way down.
Cache the outcome of `getFileStats()`, rather than repeatedly ask for it.
That includes using `isFile()`, `isDirectory()`, which are simply wrappers
around `getFileStatus()`.
Don't immediately look for a file with a `getFileStatus()` or listing call
after creating it, or try to read it immediately.
This is where eventual consistency problems surface: the data may not yet be visible.
Rely on `FileNotFoundException` being raised if the source of an operation is
missing, rather than implementing your own probe for the file before
conditionally calling the operation.
### `rename()`
Avoid any algorithm which uploads data into a temporary file and then uses
`rename()` to commit it into place with a final path.
On HDFS this offers a fast commit operation.
With S3, Wasb and other object stores, you can write straight to the destination,
knowing that the file isn't visible until you close the write: the write itself
is atomic.
The `rename()` operation may return `false` if the source is missing; this
is a weakness in the API. Consider a check before calling rename, and if/when
a new rename() call is made public, switch to it.
### `delete(path, recursive)`
Keep in mind that `delete(path, recursive)` is a no-op if the path does not exist, so
there's no need to have a check for the path existing before you call it.
`delete()` is often used as a cleanup operation.
With an object store this is slow, and may cause problems if the caller
expects an immediate response. For example, a thread may block so long
that other liveness checks start to fail.
Consider spawning off an executor thread to do these background cleanup operations.

View File

@ -14,9 +14,9 @@
# Troubleshooting
<!-- MACRO{toc|fromDepth=0|toDepth=5} -->
<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
##<a name="introduction"></a> Introduction
## <a name="introduction"></a> Introduction
Common problems working with S3 are
@ -24,28 +24,42 @@ Common problems working with S3 are
1. Authentication
1. S3 Inconsistency side-effects
Classpath is usually the first problem. For the S3x filesystem clients,
you need the Hadoop-specific filesystem clients, third party S3 client libraries
compatible with the Hadoop code, and any dependent libraries compatible with
Troubleshooting IAM Assumed Roles is covered in its
[specific documentation](assumed_roles.html#troubleshooting).
## <a name="classpath"></a> Classpath Setup
Classpath is usually the first problem. For the S3A filesystem client,
you need the Hadoop-specific filesystem clients, the very same AWS SDK library
which Hadoop was built against, and any dependent libraries compatible with
Hadoop and the specific JVM.
The classpath must be set up for the process talking to S3: if this is code
running in the Hadoop cluster, the JARs must be on that classpath. That
includes `distcp` and the `hadoop fs` command.
<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
<b>Critical:</b> *Do not attempt to "drop in" a newer version of the AWS
SDK than that which the Hadoop version was built with*
Whatever problem you have, changing the AWS SDK version will not fix things,
only change the stack traces you see.
Troubleshooting IAM Assumed Roles is covered in its
[specific documentation](assumed_roles.html#troubeshooting).
Similarly, don't try and mix a `hadoop-aws` JAR from one Hadoop release
with that of any other. The JAR must be in sync with `hadoop-common` and
some other Hadoop JARs.
## <a name="classpath"></a> Classpath Setup
<i>Randomly changing hadoop- and aws- JARs in the hope of making a problem
"go away" or to gain access to a feature you want,
will not lead to the outcome you desire.</i>
Tip: you can use [mvnrepository](http://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws)
to determine the dependency version requirements of a specific `hadoop-aws`
JAR published by the ASF.
Note that for security reasons, the S3A client does not provide much detail
on the authentication process (i.e. the secrets used to authenticate).
### `ClassNotFoundException: org.apache.hadoop.fs.s3a.S3AFileSystem`
These is Hadoop filesytem client classes, found in the `hadoop-aws` JAR.
These are Hadoop filesystem client classes, found in the `hadoop-aws` JAR.
An exception reporting this class as missing means that this JAR is not on
the classpath.
@ -56,7 +70,7 @@ the classpath.
This means that the `aws-java-sdk-bundle.jar` JAR is not on the classpath:
add it.
### Missing method in `com.amazonaws` class
### `java.lang.NoSuchMethodError` referencing a `com.amazonaws` class
This can be triggered by incompatibilities between the AWS SDK on the classpath
and the version which Hadoop was compiled with.
@ -68,6 +82,15 @@ version.
The sole fix is to use the same version of the AWS SDK with which Hadoop
was built.
This can also be caused by having more than one version of an AWS SDK
JAR on the classpath. If the full `aws-java-sdk-bundle<` JAR is on the
classpath, do not add any of the `aws-sdk-` JARs.
### `java.lang.NoSuchMethodError` referencing an `org.apache.hadoop` class
This happens if the `hadoop-aws` and `hadoop-common` JARs are out of sync.
You can't mix them around: they have to have exactly matching version numbers.
## <a name="authentication"></a> Authentication Failure
@ -115,7 +138,7 @@ mechanism.
1. If using session authentication, the session may have expired.
Generate a new session token and secret.
1. If using environement variable-based authentication, make sure that the
1. If using environment variable-based authentication, make sure that the
relevant variables are set in the environment in which the process is running.
The standard first step is: try to use the AWS command line tools with the same
@ -126,7 +149,6 @@ credentials, through a command such as:
Note the trailing "/" here; without that the shell thinks you are trying to list
your home directory under the bucket, which will only exist if explicitly created.
Attempting to list a bucket using inline credentials is a
means of verifying that the key and secret can access a bucket;
@ -186,7 +208,9 @@ Requests using the V2 API will be rejected with 400 `Bad Request`
$ bin/hadoop fs -ls s3a://frankfurt/
WARN s3a.S3AFileSystem: Client: Amazon S3 error 400: 400 Bad Request; Bad Request (retryable)
com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; Request ID: 923C5D9E75E44C06), S3 Extended Request ID: HDwje6k+ANEeDsM6aJ8+D5gUmNAMguOk2BvZ8PH3g9z0gpH+IuwT7N19oQOnIr5CIx7Vqb/uThE=
com.amazonaws.services.s3.model.AmazonS3Exception: Bad Request (Service: Amazon S3;
Status Code: 400; Error Code: 400 Bad Request; Request ID: 923C5D9E75E44C06),
S3 Extended Request ID: HDwje6k+ANEeDsM6aJ8+D5gUmNAMguOk2BvZ8PH3g9z0gpH+IuwT7N19oQOnIr5CIx7Vqb/uThE=
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
@ -231,13 +255,129 @@ As an example, the endpoint for S3 Frankfurt is `s3.eu-central-1.amazonaws.com`:
</property>
```
## <a name="access_denied"></a> `AccessDeniedException` "Access Denied"
### "403 Access denied" when trying to write data
### <a name="access_denied_unknown-ID"></a> AccessDeniedException "The AWS Access Key Id you provided does not exist in our records."
The value of `fs.s3a.access.key` does not match a known access key ID.
It may be mistyped, or the access key may have been deleted by one of the account managers.
```
java.nio.file.AccessDeniedException: bucket: doesBucketExist on bucket:
com.amazonaws.services.s3.model.AmazonS3Exception:
The AWS Access Key Id you provided does not exist in our records.
(Service: Amazon S3; Status Code: 403; Error Code: InvalidAccessKeyId;
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:214)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:260)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:314)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:256)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:231)
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:366)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:302)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3354)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:124)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3403)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3371)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:477)
at org.apache.hadoop.fs.contract.AbstractBondedFSContract.init(AbstractBondedFSContract.java:72)
at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.setup(AbstractFSContractTestBase.java:177)
at org.apache.hadoop.fs.s3a.commit.AbstractCommitITest.setup(AbstractCommitITest.java:163)
at org.apache.hadoop.fs.s3a.commit.AbstractITCommitMRJob.setup(AbstractITCommitMRJob.java:129)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
The AWS Access Key Id you provided does not exist in our records.
(Service: Amazon S3; Status Code: 403; Error Code: InvalidAccessKeyId;
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4229)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4176)
at com.amazonaws.services.s3.AmazonS3Client.getAcl(AmazonS3Client.java:3381)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1160)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1150)
at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1266)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:367)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:109)
... 27 more
```
### <a name="access_denied_disabled"></a> `AccessDeniedException` All access to this object has been disabled
Caller has no permission to access the bucket at all.
```
doesBucketExist on fdsd: java.nio.file.AccessDeniedException: fdsd: doesBucketExist on fdsd:
com.amazonaws.services.s3.model.AmazonS3Exception: All access to this object has been disabled
(Service: Amazon S3; Status Code: 403; Error Code: AllAccessDisabled; Request ID: E6229D7F8134E64F;
S3 Extended Request ID: 6SzVz2t4qa8J2Wxo/oc8yBuB13Mgrn9uMKnxVY0hsBd2kU/YdHzW1IaujpJdDXRDCQRX3f1RYn0=),
S3 Extended Request ID: 6SzVz2t4qa8J2Wxo/oc8yBuB13Mgrn9uMKnxVY0hsBd2kU/YdHzW1IaujpJdDXRDCQRX3f1RYn0=:AllAccessDisabled
All access to this object has been disabled (Service: Amazon S3; Status Code: 403;
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:205)
at org.apache.hadoop.fs.s3a.S3ALambda.once(S3ALambda.java:122)
at org.apache.hadoop.fs.s3a.S3ALambda.lambda$retry$2(S3ALambda.java:233)
at org.apache.hadoop.fs.s3a.S3ALambda.retryUntranslated(S3ALambda.java:288)
at org.apache.hadoop.fs.s3a.S3ALambda.retry(S3ALambda.java:228)
at org.apache.hadoop.fs.s3a.S3ALambda.retry(S3ALambda.java:203)
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:357)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:293)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3288)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3337)
at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3311)
at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:529)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool$BucketInfo.run(S3GuardTool.java:997)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:309)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:1218)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.main(S3GuardTool.java:1227)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: All access to this object has been disabled
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1638)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1303)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1055)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4229)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4176)
at com.amazonaws.services.s3.AmazonS3Client.getAcl(AmazonS3Client.java:3381)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1160)
at com.amazonaws.services.s3.AmazonS3Client.getBucketAcl(AmazonS3Client.java:1150)
at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1266)
at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$verifyBucketExists$1(S3AFileSystem.java:360)
at org.apache.hadoop.fs.s3a.S3ALambda.once(S3ALambda.java:120)
```
Check the name of the bucket is correct, and validate permissions for the active user/role.
### <a name="access_denied_writing"></a> `AccessDeniedException` "Access denied" when trying to manipulate data
Data can be read, but attempts to write data or manipulate the store fail with
403/Access denied.
The bucket may have an access policy which the request does not comply with.
or the caller does not have the right to access the data.
```
java.nio.file.AccessDeniedException: test/: PUT 0-byte object on test/:
@ -257,14 +397,31 @@ java.nio.file.AccessDeniedException: test/: PUT 0-byte object on test/:
```
In the AWS S3 management console, select the "permissions" tab for the bucket, then "bucket policy".
If there is no bucket policy, then the error cannot be caused by one.
If there is a bucket access policy, e.g. required encryption headers,
then the settings of the s3a client must guarantee the relevant headers are set
(e.g. the encryption options match).
Note: S3 Default Encryption options are not considered here:
if the bucket policy requires AES256 as the encryption policy on PUT requests,
then the encryption option must be set in the s3a client so that the header is set.
then the encryption option must be set in the hadoop client so that the header is set.
Otherwise, the problem will likely be that the user does not have full access to the
operation. Check what they were trying to (read vs write) and then look
at the permissions of the user/role.
If the client using [assumed roles](assumed_roles.html), and a policy
is set in `fs.s3a.assumed.role.policy`, then that policy declares
_all_ the rights which the caller has.
### <a name="kms_access_denied"></a> `AccessDeniedException` when using SSE-KMS
When trying to write or read SEE-KMS-encrypted data, the client gets a
`java.nio.AccessDeniedException` with the error 403/Forbidden.
The caller does not have the permissions to access
the key with which the data was encrypted.
## <a name="connectivity"></a> Connectivity Problems
@ -283,14 +440,14 @@ org.apache.hadoop.fs.s3a.AWSS3IOException: Received permanent redirect response
addressed using the specified endpoint. Please send all future requests to
this endpoint. (Service: Amazon S3; Status Code: 301;
Error Code: PermanentRedirect; Request ID: 7D39EC1021C61B11)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:132)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:287)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:203)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2895)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:102)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2932)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2914)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:390)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:132)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:287)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:203)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2895)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:102)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2932)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2914)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:390)
```
1. Use the [Specific endpoint of the bucket's S3 service](http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region)
@ -308,12 +465,15 @@ Using the explicit endpoint for the region is recommended for speed and
to use the V4 signing API.
### <a name="timeout"></a> "Timeout waiting for connection from pool" when writing data
### <a name="timeout_from_pool"></a> "Timeout waiting for connection from pool" when writing data
This happens when using the output stream thread pool runs out of capacity.
```
[s3a-transfer-shared-pool1-t20] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
[s3a-transfer-shared-pool1-t20] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496))
- Unable to execute HTTP request:
Timeout waiting for connection from poolorg.apache.http.conn.ConnectionPoolTimeoutException:
Timeout waiting for connection from pool
at org.apache.http.impl.conn.PoolingClientConnectionManager.leaseConnection(PoolingClientConnectionManager.java:230)
at org.apache.http.impl.conn.PoolingClientConnectionManager$1.getConnection(PoolingClientConnectionManager.java:199)
at sun.reflect.GeneratedMethodAccessor13.invoke(Unknown Source)
@ -364,6 +524,46 @@ the maximum number of allocated HTTP connections.
Set `fs.s3a.connection.maximum` to a larger value (and at least as large as
`fs.s3a.threads.max`)
### `NoHttpResponseException`
The HTTP Server did not respond.
```
2017-02-07 10:01:07,950 INFO [s3a-transfer-shared-pool1-t7] com.amazonaws.http.AmazonHttpClient:
Unable to execute HTTP request: bucket.s3.amazonaws.com:443 failed to respond
org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
at org.apache.http.impl.io.AbstractMessageParser.parse(AbstractMessageParser.java:261)
at org.apache.http.impl.AbstractHttpClientConnection.receiveResponseHeader(AbstractHttpClientConnection.java:283)
at org.apache.http.impl.conn.DefaultClientConnection.receiveResponseHeader(DefaultClientConnection.java:259)
at org.apache.http.impl.conn.ManagedClientConnectionImpl.receiveResponseHeader(ManagedClientConnectionImpl.java:209)
at org.apache.http.protocol.HttpRequestExecutor.doReceiveResponse(HttpRequestExecutor.java:272)
at com.amazonaws.http.protocol.SdkHttpRequestExecutor.doReceiveResponse(SdkHttpRequestExecutor.java:66)
at org.apache.http.protocol.HttpRequestExecutor.execute(HttpRequestExecutor.java:124)
at org.apache.http.impl.client.DefaultRequestDirector.tryExecute(DefaultRequestDirector.java:686)
at org.apache.http.impl.client.DefaultRequestDirector.execute(DefaultRequestDirector.java:488)
at org.apache.http.impl.client.AbstractHttpClient.doExecute(AbstractHttpClient.java:884)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at org.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:728)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785)
at com.amazonaws.services.s3.AmazonS3Client.copyPart(AmazonS3Client.java:1731)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:41)
at com.amazonaws.services.s3.transfer.internal.CopyPartCallable.call(CopyPartCallable.java:28)
at org.apache.hadoop.fs.s3a.SemaphoredDelegatingExecutor$CallableWithPermitRelease.call(SemaphoredDelegatingExecutor.java:222)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```
Probably network problems, unless it really is an outage of S3.
### Out of heap memory when writing with via Fast Upload
This can happen when using the upload buffering mechanism
@ -418,7 +618,8 @@ for up to date advice.
org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on test/testname/streaming/:
com.amazonaws.AmazonClientException: Failed to sanitize XML document
destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler:
Failed to sanitize XML document destined for handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
Failed to sanitize XML document destined for handler class
com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:105)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1462)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:1227)
@ -444,19 +645,136 @@ Again, we believe this is caused by the connection to S3 being broken.
It may go away if the operation is retried.
## <a name="other"></a> Other Errors
## Miscellaneous Errors
### <a name="integrity"></a> `SdkClientException` Unable to verify integrity of data upload
Something has happened to the data as it was uploaded.
```
Caused by: org.apache.hadoop.fs.s3a.AWSClientIOException: saving output on dest/_task_tmp.-ext-10000/_tmp.000000_0:
com.amazonaws.AmazonClientException: Unable to verify integrity of data upload.
Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64)
didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3.
You may need to delete the data stored in Amazon S3.
(metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null,
bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0):
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:144)
at org.apache.hadoop.fs.s3a.S3AOutputStream.close(S3AOutputStream.java:121)
at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
at org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat$1.close(HiveIgnoreKeyTextOutputFormat.java:99)
at org.apache.hadoop.hive.ql.exec.FileSinkOperator$FSPaths.closeWriters(FileSinkOperator.java:190)
... 22 more
Caused by: com.amazonaws.AmazonClientException: Unable to verify integrity of data upload.
Client calculated content hash (contentMD5: L75PalQk0CIhTp04MStVOA== in base 64)
didn't match hash (etag: 37ace01f2c383d6b9b3490933c83bb0f in hex) calculated by Amazon S3.
You may need to delete the data stored in Amazon S3.
(metadata.contentMD5: L75PalQk0CIhTp04MStVOA==, md5DigestStream: null,
bucketName: ext2, key: dest/_task_tmp.-ext-10000/_tmp.000000_0)
at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1492)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47)
... 4 more
```
As it uploads data to S3, the AWS SDK builds up an MD5 checksum of what was
PUT/POSTed. When S3 returns the checksum of the uploaded data, that is compared
with the local checksum. If there is a mismatch, this error is reported.
The uploaded data is already on S3 and will stay there, though if this happens
during a multipart upload, it may not be visible (but still billed: clean up your
multipart uploads via the `hadoop s3guard uploads` command).
Possible causes for this
1. A (possibly transient) network problem, including hardware faults.
1. A proxy server is doing bad things to the data.
1. Some signing problem, especially with third-party S3-compatible object stores.
This is a very, very rare occurrence.
If the problem is a signing one, try changing the signature algorithm.
```xml
<property>
<name>fs.s3a.signing-algorithm</name>
<value>S3SignerType</value>
</property>
```
We cannot make any promises that it will work,
only that it has been known to make the problem go away "once"
### `AWSS3IOException` The Content-MD5 you specified did not match what we received
Reads work, but writes, even `mkdir`, fail:
```
org.apache.hadoop.fs.s3a.AWSS3IOException: copyFromLocalFile(file:/tmp/hello.txt, s3a://bucket/hello.txt)
on file:/tmp/hello.txt:
The Content-MD5 you specified did not match what we received.
(Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225),
S3 Extended Request ID: null
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:127)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:69)
at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFromLocalFile(S3AFileSystem.java:1494)
at org.apache.hadoop.tools.cloudup.Cloudup.uploadOneFile(Cloudup.java:466)
at org.apache.hadoop.tools.cloudup.Cloudup.access$000(Cloudup.java:63)
at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:353)
at org.apache.hadoop.tools.cloudup.Cloudup$1.call(Cloudup.java:350)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
The Content-MD5 you specified did not match what we received.
(Service: Amazon S3; Status Code: 400; Error Code: BadDigest; Request ID: 4018131225),
S3 Extended Request ID: null
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1307)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:894)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:597)
at com.amazonaws.http.AmazonHttpClient.doExecute(AmazonHttpClient.java:363)
at com.amazonaws.http.AmazonHttpClient.executeWithTimer(AmazonHttpClient.java:329)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:308)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3659)
at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1422)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:131)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:123)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:139)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:47)
at org.apache.hadoop.fs.s3a.BlockingThreadPoolExecutorService$CallableWithPermitRelease.call(BlockingThreadPoolExecutorService.java:239)
... 4 more
```
This stack trace was seen when interacting with a third-party S3 store whose
expectations of headers related to the AWS V4 signing mechanism was not
compatible with that of the specific AWS SDK Hadoop was using.
Workaround: revert to V2 signing.
```xml
<property>
<name>fs.s3a.signing-algorithm</name>
<value>S3SignerType</value>
</property>
```
### When writing data: "java.io.FileNotFoundException: Completing multi-part upload"
A multipart upload was trying to complete, but failed as there was no upload
with that ID.
```
java.io.FileNotFoundException: Completing multi-part upload on fork-5/test/multipart/1c397ca6-9dfb-4ac1-9cf7-db666673246b:
com.amazonaws.services.s3.model.AmazonS3Exception: The specified upload does not exist.
The upload ID may be invalid, or the upload may have been aborted or completed. (Service: Amazon S3; Status Code: 404;
Error Code: NoSuchUpload;
The upload ID may be invalid, or the upload may have been aborted or completed.
(Service: Amazon S3; Status Code: 404; Error Code: NoSuchUpload;
at com.amazonaws.http.AmazonHttpClient.handleErrorResponse(AmazonHttpClient.java:1182)
at com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:770)
at com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489)
@ -482,14 +800,11 @@ for all open writes to complete the write,
### Application hangs after reading a number of files
The pool of https client connectons and/or IO threads have been used up,
The pool of https client connections and/or IO threads have been used up,
and none are being freed.
1. The pools aren't big enough. Increas `fs.s3a.connection.maximum` for
the http connections, and `fs.s3a.threads.max` for the thread pool.
1. The pools aren't big enough. See ["Timeout waiting for connection from pool"](#timeout_from_pool)
2. Likely root cause: whatever code is reading files isn't calling `close()`
on the input streams. Make sure your code does this!
And if it's someone else's: make sure you have a recent version; search their
@ -497,81 +812,13 @@ issue trackers to see if its a known/fixed problem.
If not, it's time to work with the developers, or come up with a workaround
(i.e closing the input stream yourself).
### "Timeout waiting for connection from pool"
This the same problem as above, exhibiting itself as the http connection
pool determining that it has run out of capacity.
```
java.io.InterruptedIOException: getFileStatus on s3a://example/fork-0007/test:
com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:145)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:119)
at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2040)
at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
...
Caused by: com.amazonaws.SdkClientException: Unable to execute HTTP request: Timeout waiting for connection from pool
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleRetryableException(AmazonHttpClient.java:1069)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1035)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168)
at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1249)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1162)
at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2022)
at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
...
Caused by: com.amazonaws.thirdparty.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool
at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager.leaseConnection(PoolingHttpClientConnectionManager.java:286)
at com.amazonaws.thirdparty.apache.http.impl.conn.PoolingHttpClientConnectionManager$1.get(PoolingHttpClientConnectionManager.java:263)
at sun.reflect.GeneratedMethodAccessor10.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at com.amazonaws.http.conn.ClientConnectionRequestFactory$Handler.invoke(ClientConnectionRequestFactory.java:70)
at com.amazonaws.http.conn.$Proxy15.get(Unknown Source)
at com.amazonaws.thirdparty.apache.http.impl.execchain.MainClientExec.execute(MainClientExec.java:190)
at com.amazonaws.thirdparty.apache.http.impl.execchain.ProtocolExec.execute(ProtocolExec.java:184)
at com.amazonaws.thirdparty.apache.http.impl.client.InternalHttpClient.doExecute(InternalHttpClient.java:184)
at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:82)
at com.amazonaws.thirdparty.apache.http.impl.client.CloseableHttpClient.execute(CloseableHttpClient.java:55)
at com.amazonaws.http.apache.client.impl.SdkHttpClient.execute(SdkHttpClient.java:72)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1190)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168)
at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1249)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getObjectMetadata(S3AFileSystem.java:1162)
at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:2022)
at org.apache.hadoop.fs.s3a.S3AFileSystem.checkPathForDirectory(S3AFileSystem.java:1857)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1890)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1826)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
```
This is the same problem as the previous one, exhibited differently.
### Issue: when writing data, HTTP Exceptions logged at info from `AmazonHttpClient`
```
[s3a-transfer-shared-pool4-t6] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496)) - Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
[s3a-transfer-shared-pool4-t6] INFO http.AmazonHttpClient (AmazonHttpClient.java:executeHelper(496))
- Unable to execute HTTP request: hwdev-steve-ireland-new.s3.amazonaws.com:443 failed to respond
org.apache.http.NoHttpResponseException: bucket.s3.amazonaws.com:443 failed to respond
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:143)
at org.apache.http.impl.conn.DefaultHttpResponseParser.parseHead(DefaultHttpResponseParser.java:57)
@ -606,6 +853,45 @@ will attempt to retry the operation; it may just be a transient event. If there
are many such exceptions in logs, it may be a symptom of connectivity or network
problems.
### `AWSBadRequestException` IllegalLocationConstraintException/The unspecified location constraint is incompatible
```
Cause: org.apache.hadoop.fs.s3a.AWSBadRequestException: put on :
com.amazonaws.services.s3.model.AmazonS3Exception:
The unspecified location constraint is incompatible for the region specific
endpoint this request was sent to.
(Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException;
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:178)
at org.apache.hadoop.fs.s3a.S3ALambda.execute(S3ALambda.java:64)
at org.apache.hadoop.fs.s3a.WriteOperationHelper.uploadObject(WriteOperationHelper.java:451)
at org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker.aboutToComplete(MagicCommitTracker.java:128)
at org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:373)
at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
at org.apache.hadoop.hive.ql.io.orc.WriterImpl.close(WriterImpl.java:2429)
at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:106)
at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat$OrcRecordWriter.close(OrcOutputFormat.java:91)
...
Cause: com.amazonaws.services.s3.model.AmazonS3Exception:
The unspecified location constraint is incompatible for the region specific endpoint
this request was sent to. (Service: Amazon S3; Status Code: 400; Error Code: IllegalLocationConstraintException;
Request ID: EEBC5A08BCB3A645)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
...
```
Something has been trying to write data to "/".
## File System Semantics
These are the issues where S3 does not appear to behave the way a filesystem
@ -664,7 +950,7 @@ that it is not there)
This is a visible sign of updates to the metadata server lagging
behind the state of the underlying filesystem.
Fix: Use S3Guard
Fix: Use [S3Guard](s3guard.html).
### File not visible/saved
@ -686,26 +972,74 @@ and the like. The standard strategy here is to save to HDFS and then copy to S3.
## <a name="encryption"></a> S3 Server Side Encryption
### Using SSE-KMS "Invalid arn"
### `AWSS3IOException` `KMS.NotFoundException` "Invalid arn" when using SSE-KMS
When performing file operations, the user may run into an issue where the KMS
key arn is invalid.
```
com.amazonaws.services.s3.model.AmazonS3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: 708284CF60EE233F),
S3 Extended Request ID: iHUUtXUSiNz4kv3Bdk/hf9F+wjPt8GIVvBHx/HEfCBYkn7W6zmpvbA3XT7Y5nTzcZtfuhcqDunw=:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: 708284CF60EE233F)
org.apache.hadoop.fs.s3a.AWSS3IOException: innerMkdirs on /test:
com.amazonaws.services.s3.model.AmazonS3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException;
Request ID: CA89F276B3394565),
S3 Extended Request ID: ncz0LWn8zor1cUO2fQ7gc5eyqOk3YfyQLDn2OQNoe5Zj/GqDLggUYz9QY7JhdZHdBaDTh+TL5ZQ=:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:194)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:117)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1541)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2230)
at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.mkdirs(AbstractFSContractTestBase.java:338)
at org.apache.hadoop.fs.contract.AbstractFSContractTestBase.setup(AbstractFSContractTestBase.java:193)
at org.apache.hadoop.fs.s3a.scale.S3AScaleTestBase.setup(S3AScaleTestBase.java:90)
at org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles.setup(AbstractSTestS3AHugeFiles.java:77)
at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
at org.junit.internal.runners.statements.FailOnTimeout$StatementThread.run(FailOnTimeout.java:74)
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception:
Invalid arn (Service: Amazon S3; Status Code: 400; Error Code: KMS.NotFoundException; Request ID: CA89F276B3394565)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1588)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1258)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1030)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:742)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:716)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4221)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4168)
at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1718)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:133)
at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:125)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:143)
at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:48)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```
This is due to either, the KMS key id is entered incorrectly, or the KMS key id
is in a different region than the S3 bucket being used.
Possible causes:
* the KMS key ARN is entered incorrectly, or
* the KMS key referenced by the ARN is in a different region than the S3 bucket
being used.
### Using SSE-C "Bad Request"
When performing file operations the user may run into an unexpected 400/403
error such as
```
org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/: com.amazonaws.services.s3.model.AmazonS3Exception:
org.apache.hadoop.fs.s3a.AWSS3IOException: getFileStatus on fork-4/:
com.amazonaws.services.s3.model.AmazonS3Exception:
Bad Request (Service: Amazon S3; Status Code: 400;
Error Code: 400 Bad Request; Request ID: 42F9A1987CB49A99),
S3 Extended Request ID: jU2kcwaXnWj5APB14Cgb1IKkc449gu2+dhIsW/+7x9J4D+VUkKvu78mBo03oh9jnOT2eoTLdECU=:
@ -719,83 +1053,49 @@ is used, no encryption is specified, or the SSE-C specified is incorrect.
2. A directory is encrypted with a SSE-C keyA and the user is trying to move a
file using configured SSE-C keyB into that structure.
## <a name="performance"></a> Performance
## <a name="not_all_bytes_were_read"></a> Message appears in logs "Not all bytes were read from the S3ObjectInputStream"
S3 is slower to read data than HDFS, even on virtual clusters running on
Amazon EC2.
* HDFS replicates data for faster query performance.
* HDFS stores the data on the local hard disks, avoiding network traffic
if the code can be executed on that host. As EC2 hosts often have their
network bandwidth throttled, this can make a tangible difference.
* HDFS is significantly faster for many "metadata" operations: listing
the contents of a directory, calling `getFileStatus()` on path,
creating or deleting directories. (S3Guard reduces but does not eliminate
the speed gap).
* On HDFS, Directory renames and deletes are `O(1)` operations. On
S3 renaming is a very expensive `O(data)` operation which may fail partway through
in which case the final state depends on where the copy+ delete sequence was when it failed.
All the objects are copied, then the original set of objects are deleted, so
a failure should not lose data —it may result in duplicate datasets.
* Unless fast upload enabled, the write only begins on a `close()` operation.
This can take so long that some applications can actually time out.
* File IO involving many seek calls/positioned read calls will encounter
performance problems due to the size of the HTTP requests made. Enable the
"random" fadvise policy to alleviate this at the
expense of sequential read performance and bandwidth.
This is a message which can be generated by the Amazon SDK when the client application
calls `abort()` on the HTTP input stream, rather than reading to the end of
the file/stream and causing `close()`. The S3A client does call `abort()` when
seeking round large files, [so leading to the message](https://github.com/aws/aws-sdk-java/issues/1211).
The slow performance of `rename()` surfaces during the commit phase of work,
including
No ASF Hadoop releases have shipped with an SDK which prints this message
when used by the S3A client. However third party and private builds of Hadoop
may cause the message to be logged.
* The MapReduce `FileOutputCommitter`. This also used by Apache Spark.
* DistCp's rename-after-copy operation.
* The `hdfs fs -rm` command renaming the file under `.Trash` rather than
deleting it. Use `-skipTrash` to eliminate that step.
Ignore it. The S3A client does call `abort()`, but that's because our benchmarking
shows that it is generally more efficient to abort the TCP connection and initiate
a new one than read to the end of a large file.
These operations can be significantly slower when S3 is the destination
compared to HDFS or other "real" filesystem.
Note: the threshold when data is read rather than the stream aborted can be tuned
by `fs.s3a.readahead.range`; seek policy in `fs.s3a.experimental.fadvise`.
*Improving S3 load-balancing behavior*
### <a name="no_such_bucket"></a> `FileNotFoundException` Bucket does not exist.
Amazon S3 uses a set of front-end servers to provide access to the underlying data.
The choice of which front-end server to use is handled via load-balancing DNS
service: when the IP address of an S3 bucket is looked up, the choice of which
IP address to return to the client is made based on the the current load
of the front-end servers.
The bucket does not exist.
Over time, the load across the front-end changes, so those servers considered
"lightly loaded" will change. If the DNS value is cached for any length of time,
your application may end up talking to an overloaded server. Or, in the case
of failures, trying to talk to a server that is no longer there.
And by default, for historical security reasons in the era of applets,
the DNS TTL of a JVM is "infinity".
To work with AWS better, set the DNS time-to-live of an application which
works with S3 to something lower. See [AWS documentation](http://docs.aws.amazon.com/AWSSdkDocsJava/latest/DeveloperGuide/java-dg-jvm-ttl.html).
## <a name="network_performance"></a>Troubleshooting network performance
An example of this is covered in [HADOOP-13871](https://issues.apache.org/jira/browse/HADOOP-13871).
1. For public data, use `curl`:
curl -O https://landsat-pds.s3.amazonaws.com/scene_list.gz
1. Use `nettop` to monitor a processes connections.
Consider reducing the connection timeout of the s3a connection.
```xml
<property>
<name>fs.s3a.connection.timeout</name>
<value>15000</value>
</property>
```
This *may* cause the client to react faster to network pauses, so display
stack traces fast. At the same time, it may be less resilient to
connectivity problems.
java.io.FileNotFoundException: Bucket stevel45r56666 does not exist
at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:361)
at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:293)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3288)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:123)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3337)
at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:3311)
at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:529)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool$BucketInfo.run(S3GuardTool.java:997)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:309)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.run(S3GuardTool.java:1218)
at org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.main(S3GuardTool.java:1227)
```
Check the URI. If using a third-party store, verify that you've configured
the client to talk to the specific server in `fs.s3a.endpoint`.
## Other Issues
### <a name="logging"></a> Enabling low-level logging
@ -816,7 +1116,7 @@ log4j.logger.org.apache.http=DEBUG
```
This produces a log such as this, wich is for a V4-authenticated PUT of a 0-byte file used
This produces a log such as this, which is for a V4-authenticated PUT of a 0-byte file used
as an empty directory marker
```
@ -866,9 +1166,9 @@ execchain.MainClientExec (MainClientExec.java:execute(284)) - Connection can be
## <a name="retries"></a> Reducing failures by configuring retry policy
The S3A client can ba configured to rety those operations which are considered
retriable. That can be because they are idempotent, or
because there failure happened before the request was processed by S3.
The S3A client can ba configured to retry those operations which are considered
retryable. That can be because they are idempotent, or
because the failure happened before the request was processed by S3.
The number of retries and interval between each retry can be configured:
@ -893,8 +1193,8 @@ Not all failures are retried. Specifically excluded are those considered
unrecoverable:
* Low-level networking: `UnknownHostException`, `NoRouteToHostException`.
* 302 redirects
* Missing resources, 404/`FileNotFoundException`
* 302 redirects.
* Missing resources, 404/`FileNotFoundException`.
* HTTP 416 response/`EOFException`. This can surface if the length of a file changes
while another client is reading it.
* Failures during execution or result processing of non-idempotent operations where
@ -910,79 +1210,6 @@ be idempotent, and will retry them on failure. These are only really idempotent
if no other client is attempting to manipulate the same objects, such as:
renaming() the directory tree or uploading files to the same location.
Please don't do that. Given that the emulated directory rename and delete operations
aren't atomic, even without retries, multiple S3 clients working with the same
are not atomic, even without retries, multiple S3 clients working with the same
paths can interfere with each other
#### <a name="retries"></a> Throttling
When many requests are made of a specific S3 bucket (or shard inside it),
S3 will respond with a 503 "throttled" response.
Throttling can be recovered from, provided overall load decreases.
Furthermore, because it is sent before any changes are made to the object store,
is inherently idempotent. For this reason, the client will always attempt to
retry throttled requests.
The limit of the number of times a throttled request can be retried,
and the exponential interval increase between attempts, can be configured
independently of the other retry limits.
```xml
<property>
<name>fs.s3a.retry.throttle.limit</name>
<value>20</value>
<description>
Number of times to retry any throttled request.
</description>
</property>
<property>
<name>fs.s3a.retry.throttle.interval</name>
<value>500ms</value>
<description>
Interval between retry attempts on throttled requests.
</description>
</property>
```
If a client is failing due to `AWSServiceThrottledException` failures,
increasing the interval and limit *may* address this. However, it
it is a sign of AWS services being overloaded by the sheer number of clients
and rate of requests. Spreading data across different buckets, and/or using
a more balanced directory structure may be beneficial.
Consult [the AWS documentation](http://docs.aws.amazon.com/AmazonS3/latest/dev/request-rate-perf-considerations.html).
Reading or writing data encrypted with SSE-KMS forces S3 to make calls of
the AWS KMS Key Management Service, which comes with its own
[Request Rate Limits](http://docs.aws.amazon.com/kms/latest/developerguide/limits.html).
These default to 1200/second for an account, across all keys and all uses of
them, which, for S3 means: across all buckets with data encrypted with SSE-KMS.
###### Tips to Keep Throttling down
* If you are seeing a lot of throttling responses on a large scale
operation like a `distcp` copy, *reduce* the number of processes trying
to work with the bucket (for distcp: reduce the number of mappers with the
`-m` option).
* If you are reading or writing lists of files, if you can randomize
the list so they are not processed in a simple sorted order, you may
reduce load on a specific shard of S3 data, so potentially increase throughput.
* An S3 Bucket is throttled by requests coming from all
simultaneous clients. Different applications and jobs may interfere with
each other: consider that when troubleshooting.
Partitioning data into different buckets may help isolate load here.
* If you are using data encrypted with SSE-KMS, then the
will also apply: these are stricter than the S3 numbers.
If you believe that you are reaching these limits, you may be able to
get them increased.
Consult [the KMS Rate Limit documentation](http://docs.aws.amazon.com/kms/latest/developerguide/limits.html).
* S3Guard uses DynamoDB for directory and file lookups;
it is rate limited to the amount of (guaranteed) IO purchased for a
table. If significant throttling events/rate is observed here, the preallocated
IOPs can be increased with the `s3guard set-capacity` command, or
through the AWS Console. Throttling events in S3Guard are noted in logs, and
also in the S3A metrics `s3guard_metadatastore_throttle_rate` and
`s3guard_metadatastore_throttled`.