HADOOP-11261 Set custom endpoint for S3A. (Thomas Demoor via stevel)

This commit is contained in:
Steve Loughran 2015-01-16 10:15:22 +00:00
parent 5d1cca34fa
commit 000ca83ea3
4 changed files with 110 additions and 1 deletions

View File

@ -486,6 +486,8 @@ Release 2.7.0 - UNRELEASED
HADOOP-8757. Metrics should disallow names with invalid characters
(rchiang via rkanter)
HADOOP-11261 Set custom endpoint for S3A. (Thomas Demoor via stevel)
OPTIMIZATIONS
HADOOP-11323. WritableComparator#compare keeps reference to byte array.

View File

@ -28,7 +28,10 @@ public class Constants {
// connect to s3 over ssl?
public static final String SECURE_CONNECTIONS = "fs.s3a.connection.ssl.enabled";
public static final boolean DEFAULT_SECURE_CONNECTIONS = true;
//use a custom endpoint?
public static final String ENDPOINT = "fs.s3a.endpoint";
// number of times we should retry errors
public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum";
public static final int DEFAULT_MAX_ERROR_RETRIES = 10;

View File

@ -57,6 +57,7 @@ import com.amazonaws.services.s3.transfer.Upload;
import com.amazonaws.event.ProgressListener;
import com.amazonaws.event.ProgressEvent;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
@ -176,6 +177,16 @@ public class S3AFileSystem extends FileSystem {
DEFAULT_SOCKET_TIMEOUT));
s3 = new AmazonS3Client(credentials, awsConf);
String endPoint = conf.getTrimmed(ENDPOINT,"");
if (!endPoint.isEmpty()) {
try {
s3.setEndpoint(endPoint);
} catch (IllegalArgumentException e) {
String msg = "Incorrect endpoint: " + e.getMessage();
LOG.error(msg);
throw new IllegalArgumentException(msg, e);
}
}
maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS);
partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
@ -262,6 +273,14 @@ public class S3AFileSystem extends FileSystem {
return uri;
}
/**
* Returns the S3 client used by this filesystem.
* @return AmazonS3Client
*/
@VisibleForTesting
AmazonS3Client getAmazonS3Client() {
return s3;
}
public S3AFileSystem() {
super();

View File

@ -0,0 +1,85 @@
/**
* 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 com.amazonaws.services.s3.AmazonS3Client;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
public class TestS3AConfiguration {
private Configuration conf;
private S3AFileSystem fs;
private static final Logger LOG =
LoggerFactory.getLogger(TestS3AConfiguration.class);
private static final String TEST_ENDPOINT = "test.fs.s3a.endpoint";
@Rule
public Timeout testTimeout = new Timeout(30 * 60 * 1000);
/**
* Test if custom endpoint is picked up.
* <p/>
* The test expects TEST_ENDPOINT to be defined in the Configuration
* describing the endpoint of the bucket to which TEST_FS_S3A_NAME points
* (f.i. "s3-eu-west-1.amazonaws.com" if the bucket is located in Ireland).
* Evidently, the bucket has to be hosted in the region denoted by the
* endpoint for the test to succeed.
* <p/>
* More info and the list of endpoint identifiers:
* http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region
*
* @throws Exception
*/
@Test
public void TestEndpoint() throws Exception {
conf = new Configuration();
String endpoint = conf.getTrimmed(TEST_ENDPOINT, "");
if (endpoint.isEmpty()) {
LOG.warn("Custom endpoint test skipped as " + TEST_ENDPOINT + "config " +
"setting was not detected");
} else {
conf.set(Constants.ENDPOINT, endpoint);
fs = S3ATestUtils.createTestFileSystem(conf);
AmazonS3Client s3 = fs.getAmazonS3Client();
String endPointRegion = "";
// Differentiate handling of "s3-" and "s3." based endpoint identifiers
String[] endpointParts = StringUtils.split(endpoint, '.');
if (endpointParts.length == 3) {
endPointRegion = endpointParts[0].substring(3);
} else if (endpointParts.length == 4) {
endPointRegion = endpointParts[1];
} else {
fail("Unexpected endpoint");
}
assertEquals("Endpoint config setting and bucket location differ: ",
endPointRegion, s3.getBucketLocation(fs.getUri().getHost()));
}
}
}