HADOOP-13674. S3A can provide a more detailed error message when accessing a bucket through an incorrect S3 endpoint. Contributed by Chris Nauroth.

This commit is contained in:
Chris Nauroth 2016-10-04 10:36:58 -07:00
parent 382307cbdd
commit 88b9444a81
4 changed files with 162 additions and 79 deletions

View File

@ -48,6 +48,7 @@
import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
/**
@ -64,6 +65,7 @@ public final class S3AUtils {
= "instantiation exception";
static final String NOT_AWS_PROVIDER =
"does not implement AWSCredentialsProvider";
static final String ENDPOINT_KEY = "Endpoint";
private S3AUtils() {
}
@ -117,6 +119,21 @@ public static IOException translateException(String operation,
int status = ase.getStatusCode();
switch (status) {
case 301:
if (s3Exception != null) {
if (s3Exception.getAdditionalDetails() != null &&
s3Exception.getAdditionalDetails().containsKey(ENDPOINT_KEY)) {
message = String.format("Received permanent redirect response to "
+ "endpoint %s. This likely indicates that the S3 endpoint "
+ "configured in %s does not match the AWS region containing "
+ "the bucket.",
s3Exception.getAdditionalDetails().get(ENDPOINT_KEY), ENDPOINT);
}
ioe = new AWSS3IOException(message, s3Exception);
} else {
ioe = new AWSServiceIOException(message, ase);
}
break;
// permissions
case 401:
case 403:

View File

@ -1184,33 +1184,27 @@ As an example, the endpoint for S3 Frankfurt is `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
This surfaces when `fs.s3a.endpoint` is configured to use an S3 service endpoint
which is neither the original AWS one, `s3.amazonaws.com` , nor the one where
the bucket is hosted.
the bucket is hosted. The error message contains the redirect target returned
by S3, which can be used to determine the correct value for `fs.s3a.endpoint`.
```
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)
org.apache.hadoop.fs.s3a.AWSS3IOException: Received permanent redirect response
to bucket.s3-us-west-2.amazonaws.com. This likely indicates that the S3
endpoint configured in fs.s3a.endpoint does not match the AWS region
containing the bucket.: 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: 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)
```
1. Use the [Specific endpoint of the bucket's S3 service](http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region)

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.fs.s3a;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -34,7 +31,6 @@
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.nio.file.AccessDeniedException;
import java.util.concurrent.Callable;
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
@ -138,55 +134,4 @@ private void assertIsEOF(String operation, int readResult) {
assertEquals("Expected EOF from "+ operation
+ "; got char " + (char) readResult, -1, readResult);
}
@Test
public void test404isNotFound() throws Throwable {
verifyTranslated(FileNotFoundException.class, createS3Exception(404));
}
protected Exception verifyTranslated(Class clazz,
AmazonClientException exception) throws Exception {
return verifyExceptionClass(clazz,
translateException("test", "/", exception));
}
@Test
public void test401isNotPermittedFound() throws Throwable {
verifyTranslated(AccessDeniedException.class,
createS3Exception(401));
}
protected AmazonS3Exception createS3Exception(int code) {
AmazonS3Exception source = new AmazonS3Exception("");
source.setStatusCode(code);
return source;
}
@Test
public void testGenericS3Exception() throws Throwable {
// S3 exception of no known type
AWSS3IOException ex = (AWSS3IOException)verifyTranslated(
AWSS3IOException.class,
createS3Exception(451));
assertEquals(451, ex.getStatusCode());
}
@Test
public void testGenericServiceS3Exception() throws Throwable {
// service exception of no known type
AmazonServiceException ase = new AmazonServiceException("unwind");
ase.setStatusCode(500);
AWSServiceIOException ex = (AWSServiceIOException)verifyTranslated(
AWSServiceIOException.class,
ase);
assertEquals(500, ex.getStatusCode());
}
@Test
public void testGenericClientException() throws Throwable {
// Generic Amazon exception
verifyTranslated(AWSClientIOException.class,
new AmazonClientException(""));
}
}

View File

@ -0,0 +1,127 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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.
*/
package org.apache.hadoop.fs.s3a;
import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.fs.s3a.S3AUtils.*;
import static org.junit.Assert.*;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.nio.file.AccessDeniedException;
import java.util.Collections;
import java.util.Map;
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import org.junit.Test;
/**
* Unit test suite covering translation of AWS SDK exceptions to S3A exceptions.
*/
public class TestS3AExceptionTranslation {
@Test
public void test301ContainsEndpoint() throws Exception {
AmazonS3Exception s3Exception = createS3Exception("wrong endpoint", 301,
Collections.singletonMap(S3AUtils.ENDPOINT_KEY,
"bucket.s3-us-west-2.amazonaws.com"));
AWSS3IOException ex = (AWSS3IOException)verifyTranslated(
AWSS3IOException.class, s3Exception);
assertEquals(301, ex.getStatusCode());
assertNotNull(ex.getMessage());
assertTrue(ex.getMessage().contains("bucket.s3-us-west-2.amazonaws.com"));
assertTrue(ex.getMessage().contains(ENDPOINT));
}
@Test
public void test401isNotPermittedFound() throws Exception {
verifyTranslated(AccessDeniedException.class,
createS3Exception(401));
}
@Test
public void test403isNotPermittedFound() throws Exception {
verifyTranslated(AccessDeniedException.class,
createS3Exception(403));
}
@Test
public void test404isNotFound() throws Exception {
verifyTranslated(FileNotFoundException.class, createS3Exception(404));
}
@Test
public void test410isNotFound() throws Exception {
verifyTranslated(FileNotFoundException.class, createS3Exception(410));
}
@Test
public void test416isEOF() throws Exception {
verifyTranslated(EOFException.class, createS3Exception(416));
}
@Test
public void testGenericS3Exception() throws Exception {
// S3 exception of no known type
AWSS3IOException ex = (AWSS3IOException)verifyTranslated(
AWSS3IOException.class,
createS3Exception(451));
assertEquals(451, ex.getStatusCode());
}
@Test
public void testGenericServiceS3Exception() throws Exception {
// service exception of no known type
AmazonServiceException ase = new AmazonServiceException("unwind");
ase.setStatusCode(500);
AWSServiceIOException ex = (AWSServiceIOException)verifyTranslated(
AWSServiceIOException.class,
ase);
assertEquals(500, ex.getStatusCode());
}
@Test
public void testGenericClientException() throws Exception {
// Generic Amazon exception
verifyTranslated(AWSClientIOException.class,
new AmazonClientException(""));
}
private static AmazonS3Exception createS3Exception(int code) {
return createS3Exception("", code, null);
}
private static AmazonS3Exception createS3Exception(String message, int code,
Map<String, String> additionalDetails) {
AmazonS3Exception source = new AmazonS3Exception(message);
source.setStatusCode(code);
source.setAdditionalDetails(additionalDetails);
return source;
}
private static Exception verifyTranslated(Class clazz,
AmazonClientException exception) throws Exception {
return verifyExceptionClass(clazz,
translateException("test", "/", exception));
}
}