diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index 01a2bae1a2..cb1df83d5c 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -634,6 +634,60 @@ this capability. any call to setReadahead() is made to an open stream. +### Working with buckets in different regions + +S3 Buckets are hosted in different regions, the default being US-East. +The client talks to it by default, under the URL `s3.amazonaws.com` + +S3A can work with buckets from any region. Each region has its own +S3 endpoint, documented [by Amazon](http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region). + +1. Applications running in EC2 infrastructure do not pay for IO to/from +*local S3 buckets*. They will be billed for access to remote buckets. Always +use local buckets and local copies of data, wherever possible. +1. The default S3 endpoint can support data IO with any bucket when the V1 request +signing protocol is used. +1. When the V4 signing protocol is used, AWS requires the explicit region endpoint +to be used —hence S3A must be configured to use the specific endpoint. This +is done in the configuration option `fs.s3a.endpoint`. +1. All endpoints other than the default endpoint only support interaction +with buckets local to that S3 instance. + +While it is generally simpler to use the default endpoint, working with +V4-signing-only regions (Frankfurt, Seoul) requires the endpoint to be identified. +Expect better performance from direct connections —traceroute will give you some insight. + +Examples: + +The default endpoint: + +```xml + + fs.s3a.endpoint + s3.amazonaws.com + +``` + +Frankfurt + +```xml + + fs.s3a.endpoint + s3.eu-central-1.amazonaws.com + +``` + +Seoul +```xml + + fs.s3a.endpoint + s3.ap-northeast-2.amazonaws.com + +``` + +If the wrong endpoint is used, the request may fail. This may be reported as a 301/redirect error, +or as a 400 Bad Request. + ### S3AFastOutputStream **Warning: NEW in hadoop 2.7. UNSTABLE, EXPERIMENTAL: use at own risk** @@ -819,8 +873,6 @@ of environment-variable authentication by attempting to use the `hdfs fs` comman to read or write data on S3. That is: comment out the `fs.s3a` secrets and rely on the environment variables. -S3 Frankfurt is a special case. It uses the V4 authentication API. - ### Authentication failures running on Java 8u60+ A change in the Java 8 JVM broke some of the `toString()` string generation @@ -829,6 +881,106 @@ generate authentication headers suitable for validation by S3. Fix: make sure that the version of Joda Time is 2.8.1 or later. +### "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or elsewhere + + +S3 Frankfurt and Seoul *only* support +[the V4 authentication API](http://docs.aws.amazon.com/AmazonS3/latest/API/sig-v4-authenticating-requests.html). + +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= + 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) + 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.headBucket(AmazonS3Client.java:1107) + at com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070) + at org.apache.hadoop.fs.s3a.S3AFileSystem.verifyBucketExists(S3AFileSystem.java:307) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:284) + at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2793) + at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:101) + at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2830) + at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2812) + at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:389) + at org.apache.hadoop.fs.Path.getFileSystem(Path.java:356) + at org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:325) + at org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235) + at org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218) + at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:103) + at org.apache.hadoop.fs.shell.Command.run(Command.java:165) + at org.apache.hadoop.fs.FsShell.run(FsShell.java:315) + at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) + at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90) + at org.apache.hadoop.fs.FsShell.main(FsShell.java:373) +ls: doesBucketExist on frankfurt-new: com.amazonaws.services.s3.model.AmazonS3Exception: + Bad Request (Service: Amazon S3; Status Code: 400; Error Code: 400 Bad Request; +``` + +This happens when trying to work with any S3 service which only supports the +"V4" signing API —and he client is configured to use the default S3A service +endpoint. + +The S3A client needs to be given the endpoint to use via the `fs.s3a.endpoint` +property. + +```xml + + fs.s3a.endpoint + s3.eu-central-1.amazonaws.com + +``` + +### Error message "The bucket you are attempting to access must be addressed using the specified endpoint" + +This surfaces when `fs.s3a.endpoint` is configured to use S3 service endpoint +which is neither the original AWS one, `s3.amazonaws.com` , nor the one where +the bucket is hosted. + +``` +org.apache.hadoop.fs.s3a.AWSS3IOException: purging multipart uploads on landsat-pds: + com.amazonaws.services.s3.model.AmazonS3Exception: + The bucket you are attempting to access must be addressed using the specified endpoint. + Please send all future requests to this endpoint. + (Service: Amazon S3; Status Code: 301; Error Code: PermanentRedirect; Request ID: 5B7A5D18BE596E4B), + S3 Extended Request ID: uE4pbbmpxi8Nh7rycS6GfIEi9UH/SWmJfGtM9IeKvRyBPZp/hN7DbPyz272eynz3PEMM2azlhjE=: + + 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) + 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.invoke(AmazonS3Client.java:3738) + at com.amazonaws.services.s3.AmazonS3Client.listMultipartUploads(AmazonS3Client.java:2796) + at com.amazonaws.services.s3.transfer.TransferManager.abortMultipartUploads(TransferManager.java:1217) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initMultipartUploads(S3AFileSystem.java:454) + at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:289) + at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2715) + at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:96) + at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2749) + at org.apache.hadoop.fs.FileSystem$Cache.getUnique(FileSystem.java:2737) + at org.apache.hadoop.fs.FileSystem.newInstance(FileSystem.java:430) +``` + +1. Use the [Specific endpoint of the bucket's S3 service](http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region) +1. If not using "V4" authentication (see above), the original S3 endpoint +can be used: + +``` + + fs.s3a.endpoint + s3.amazonaws.com + +``` + +Using the explicit endpoint for the region is recommended for speed and the +ability to use the V4 signing API. + ## Visible S3 Inconsistency Amazon S3 is *an eventually consistent object store*. That is: not a filesystem. @@ -1124,7 +1276,43 @@ that the file `contract-test-options.xml` does not contain any secret credentials itself. As the auth keys XML file is kept out of the source code tree, it is not going to get accidentally committed. -### Running Tests against non-AWS storage infrastructures + +### Running the Tests + +After completing the configuration, execute the test run through Maven. + + mvn clean test + +It's also possible to execute multiple test suites in parallel by enabling the +`parallel-tests` Maven profile. The tests spend most of their time blocked on +network I/O with the S3 service, so running in parallel tends to complete full +test runs faster. + + mvn -Pparallel-tests clean test + +Some tests must run with exclusive access to the S3 bucket, so even with the +`parallel-tests` profile enabled, several test suites will run in serial in a +separate Maven execution step after the parallel tests. + +By default, the `parallel-tests` profile runs 4 test suites concurrently. This +can be tuned by passing the `testsThreadCount` argument. + + mvn -Pparallel-tests -DtestsThreadCount=8 clean test + +### Testing against different regions + +S3A can connect to different regions —the tests support this. Simply +define the target region in `contract-tests.xml` or any `auth-keys.xml` +file referenced. + +```xml + + fs.s3a.endpoint + s3.eu-central-1.amazonaws.com + +``` +This is used for all tests expect for scale tests using a Public CSV.gz file +(see below) ### S3A session tests @@ -1149,14 +1337,14 @@ An alternate endpoint may be defined in `test.fs.s3a.sts.endpoint`. The default is ""; meaning "use the amazon default value". -#### CSV Data source +#### CSV Data source Tests The `TestS3AInputStreamPerformance` tests require read access to a multi-MB text file. The default file for these tests is one published by amazon, [s3a://landsat-pds.s3.amazonaws.com/scene_list.gz](http://landsat-pds.s3.amazonaws.com/scene_list.gz). This is a gzipped CSV index of other files which amazon serves for open use. -The path to this object is set in the option `fs.s3a.scale.test.csvfile`: +The path to this object is set in the option `fs.s3a.scale.test.csvfile`, fs.s3a.scale.test.csvfile @@ -1165,21 +1353,37 @@ The path to this object is set in the option `fs.s3a.scale.test.csvfile`: 1. If the option is not overridden, the default value is used. This is hosted in Amazon's US-east datacenter. -1. If the property is empty, tests which require it will be skipped. +1. If `fs.s3a.scale.test.csvfile` is empty, tests which require it will be skipped. 1. If the data cannot be read for any reason then the test will fail. 1. If the property is set to a different path, then that data must be readable and "sufficiently" large. To test on different S3 endpoints, or alternate infrastructures supporting -the same APIs, the option `fs.s3a.scale.test.csvfile` must therefore be +the same APIs, the option `fs.s3a.scale.test.csvfile` must either be set to " ", or an object of at least 10MB is uploaded to the object store, and the `fs.s3a.scale.test.csvfile` option set to its path. - - fs.s3a.scale.test.csvfile - - +```xml + + fs.s3a.scale.test.csvfile + + +``` +(the reason the space or newline is needed is to add "an empty entry"; an empty +`` would be considered undefined and pick up the default) + +*Note:* if using a test file in an S3 region requiring a different endpoint value +set in `fs.s3a.endpoint`, define it in `fs.s3a.scale.test.csvfile.endpoint`. +If the default CSV file is used, the tests will automatically use the us-east +endpoint: + +```xml + + fs.s3a.scale.test.csvfile.endpoint + s3.amazonaws.com + +``` #### Scale test operation count @@ -1218,27 +1422,6 @@ smaller to achieve faster test runs. 10240 -### Running the Tests - -After completing the configuration, execute the test run through Maven. - - mvn clean test - -It's also possible to execute multiple test suites in parallel by enabling the -`parallel-tests` Maven profile. The tests spend most of their time blocked on -network I/O with the S3 service, so running in parallel tends to complete full -test runs faster. - - mvn -Pparallel-tests clean test - -Some tests must run with exclusive access to the S3 bucket, so even with the -`parallel-tests` profile enabled, several test suites will run in serial in a -separate Maven execution step after the parallel tests. - -By default, the `parallel-tests` profile runs 4 test suites concurrently. This -can be tuned by passing the `testsThreadCount` argument. - - mvn -Pparallel-tests -DtestsThreadCount=8 clean test ### Testing against non AWS S3 endpoints. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 04010d635b..39c60287d0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -21,6 +21,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.s3a.scale.S3AScaleTestBase; import org.junit.Assert; import org.junit.internal.AssumptionViolatedException; import org.slf4j.Logger; @@ -133,6 +134,20 @@ public static void eventually(int timeout, Callable callback) throw lastException; } + /** + * patch the endpoint option so that irrespective of where other tests + * are working, the IO performance tests can work with the landsat + * images. + * @param conf configuration to patch + */ + public static void useCSVDataEndpoint(Configuration conf) { + String endpoint = conf.getTrimmed(S3AScaleTestBase.KEY_CSVTEST_ENDPOINT, + S3AScaleTestBase.DEFAULT_CSVTEST_ENDPOINT); + if (!endpoint.isEmpty()) { + conf.set(ENDPOINT, endpoint); + } + } + /** * The exception to raise so as to exit fast from * {@link #eventually(int, Callable)}. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java index a25ca9cd36..4ff84079fd 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java @@ -133,6 +133,7 @@ public void testAnonymousProvider() throws Exception { AnonymousAWSCredentialsProvider.class.getName()); Path testFile = new Path( conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE)); + S3ATestUtils.useCSVDataEndpoint(conf); FileSystem fs = FileSystem.newInstance(testFile.toUri(), conf); assertNotNull(fs); assertTrue(fs instanceof S3AFileSystem); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java index 21639b151e..d861a16230 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java @@ -82,13 +82,26 @@ public static void nameThread() { */ public static final String KEY_CSVTEST_FILE = S3A_SCALE_TEST + "csvfile"; - /** * Default path for the multi MB test file: {@value}. */ public static final String DEFAULT_CSVTEST_FILE = "s3a://landsat-pds/scene_list.gz"; + /** + * Endpoint for the S3 CSV/scale tests. This defaults to + * being us-east. + */ + public static final String KEY_CSVTEST_ENDPOINT = + S3A_SCALE_TEST + "csvfile.endpoint"; + + /** + * Endpoint for the S3 CSV/scale tests. This defaults to + * being us-east. + */ + public static final String DEFAULT_CSVTEST_ENDPOINT = + "s3.amazonaws.com"; + /** * The default number of operations to perform: {@value}. */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java index bddd8e26c8..d6d9d66745 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/TestS3AInputStreamPerformance.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3AInputStream; import org.apache.hadoop.fs.s3a.S3AInstrumentation; +import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CompressionCodec; @@ -79,6 +80,7 @@ public void openFS() throws IOException { assumptionMessage = "Empty test property: " + KEY_CSVTEST_FILE; testDataAvailable = false; } else { + S3ATestUtils.useCSVDataEndpoint(conf); testData = new Path(testFile); Path path = this.testData; bindS3aFS(path);