Sfoglia il codice sorgente

HADOOP-18908. Improve S3A region handling. (#6187)

S3A region logic improved for better inference and
to be compatible with previous releases

1. If you are using an AWS S3 AccessPoint, its region is determined
   from the ARN itself.
2. If fs.s3a.endpoint.region is set and non-empty, it is used.
3. If fs.s3a.endpoint is an s3.*.amazonaws.com url, 
   the region is determined by by parsing the URL 
   Note: vpce endpoints are not handled by this.
4. If fs.s3a.endpoint.region==null, and none could be determined
   from the endpoint, use us-east-2 as default.
5. If fs.s3a.endpoint.region=="" then it is handed off to
   The default AWS SDK resolution process.

Consult the AWS SDK documentation for the details on its resolution
process, knowing that it is complicated and may use environment variables,
entries in ~/.aws/config, IAM instance information within
EC2 deployments and possibly even JSON resources on the classpath.
Put differently: it is somewhat brittle across deployments.

Contributed by Ahmar Suhail
Steve Loughran 1 anno fa
parent
commit
e0563fed50

+ 0 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java

@@ -407,10 +407,6 @@ public final class StoreStatisticNames {
   public static final String MULTIPART_UPLOAD_LIST
       = "multipart_upload_list";
 
-  /** Probe for store region: {@value}. */
-  public static final String STORE_REGION_PROBE
-      = "store_region_probe";
-
   private StoreStatisticNames() {
   }
 

+ 6 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java

@@ -1179,6 +1179,12 @@ public final class Constants {
    */
   public static final String AWS_S3_CENTRAL_REGION = "us-east-1";
 
+  /**
+   * The default S3 region when using cross region client.
+   * Value {@value}.
+   */
+  public static final String AWS_S3_DEFAULT_REGION = "us-east-2";
+
   /**
    * Require that all S3 access is made through Access Points.
    */

+ 105 - 10
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java

@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.s3a.impl.AWSClientConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import software.amazon.awssdk.awscore.util.AwsHostNameUtils;
 import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
 import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
 import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
@@ -48,6 +49,9 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
 
+import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_DEFAULT_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT;
 import static org.apache.hadoop.fs.s3a.impl.AWSHeaders.REQUESTER_PAYS_HEADER;
 import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS;
 import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS;
@@ -66,12 +70,27 @@ public class DefaultS3ClientFactory extends Configured
 
   private static final String REQUESTER_PAYS_HEADER_VALUE = "requester";
 
+  private static final String S3_SERVICE_NAME = "s3";
+
   /**
    * Subclasses refer to this.
    */
   protected static final Logger LOG =
       LoggerFactory.getLogger(DefaultS3ClientFactory.class);
 
+  /**
+   * A one-off warning of default region chains in use.
+   */
+  private static final LogExactlyOnce WARN_OF_DEFAULT_REGION_CHAIN =
+      new LogExactlyOnce(LOG);
+
+  /**
+   * Warning message printed when the SDK Region chain is in use.
+   */
+  private static final String SDK_REGION_CHAIN_IN_USE =
+      "S3A filesystem client is using"
+          + " the SDK region resolution chain.";
+
 
   /** Exactly once log to inform about ignoring the AWS-SDK Warnings for CSE. */
   private static final LogExactlyOnce IGNORE_CSE_WARN = new LogExactlyOnce(LOG);
@@ -138,15 +157,7 @@ public class DefaultS3ClientFactory extends Configured
       BuilderT builder, S3ClientCreationParameters parameters, Configuration conf, String bucket)
       throws IOException {
 
-    Region region = parameters.getRegion();
-    LOG.debug("Using region {}", region);
-
-    URI endpoint = getS3Endpoint(parameters.getEndpoint(), conf);
-
-    if (endpoint != null) {
-      builder.endpointOverride(endpoint);
-      LOG.debug("Using endpoint {}", endpoint);
-    }
+    configureEndpointAndRegion(builder, parameters, conf);
 
     S3Configuration serviceConfiguration = S3Configuration.builder()
             .pathStyleAccessEnabled(parameters.isPathStyleAccess())
@@ -155,7 +166,6 @@ public class DefaultS3ClientFactory extends Configured
     return builder
         .overrideConfiguration(createClientOverrideConfiguration(parameters, conf))
         .credentialsProvider(parameters.getCredentialSet())
-        .region(region)
         .serviceConfiguration(serviceConfiguration);
   }
 
@@ -201,6 +211,72 @@ public class DefaultS3ClientFactory extends Configured
     return clientOverrideConfigBuilder.build();
   }
 
+  /**
+   * This method configures the endpoint and region for a S3 client.
+   * The order of configuration is:
+   *
+   * <ol>
+   * <li>If region is configured via fs.s3a.endpoint.region, use it.</li>
+   * <li>If endpoint is configured via via fs.s3a.endpoint, set it.
+   *     If no region is configured, try to parse region from endpoint. </li>
+   * <li> If no region is configured, and it could not be parsed from the endpoint,
+   *     set the default region as US_EAST_2 and enable cross region access. </li>
+   * <li> If configured region is empty, fallback to SDK resolution chain. </li>
+   * </ol>
+   *
+   * @param builder S3 client builder.
+   * @param parameters parameter object
+   * @param conf  conf configuration object
+   * @param <BuilderT> S3 client builder type
+   * @param <ClientT> S3 client type
+   */
+  private <BuilderT extends S3BaseClientBuilder<BuilderT, ClientT>, ClientT> void configureEndpointAndRegion(
+      BuilderT builder, S3ClientCreationParameters parameters, Configuration conf) {
+    URI endpoint = getS3Endpoint(parameters.getEndpoint(), conf);
+
+    String configuredRegion = parameters.getRegion();
+    Region region = null;
+    String origin = "";
+
+    // If the region was configured, set it.
+    if (configuredRegion != null && !configuredRegion.isEmpty()) {
+      origin = AWS_REGION;
+      region = Region.of(configuredRegion);
+    }
+
+    if (endpoint != null) {
+      builder.endpointOverride(endpoint);
+      // No region was configured, try to determine it from the endpoint.
+      if (region == null) {
+        region = getS3RegionFromEndpoint(parameters.getEndpoint());
+        if (region != null) {
+          origin = "endpoint";
+        }
+      }
+      LOG.debug("Setting endpoint to {}", endpoint);
+    }
+
+    if (region != null) {
+      builder.region(region);
+    } else if (configuredRegion == null) {
+      // no region is configured, and none could be determined from the endpoint.
+      // Use US_EAST_2 as default.
+      region = Region.of(AWS_S3_DEFAULT_REGION);
+      builder.crossRegionAccessEnabled(true);
+      builder.region(region);
+      origin = "cross region access fallback";
+    } else if (configuredRegion.isEmpty()) {
+      // region configuration was set to empty string.
+      // allow this if people really want it; it is OK to rely on this
+      // when deployed in EC2.
+      WARN_OF_DEFAULT_REGION_CHAIN.warn(SDK_REGION_CHAIN_IN_USE);
+      LOG.debug(SDK_REGION_CHAIN_IN_USE);
+      origin = "SDK region chain";
+    }
+
+    LOG.debug("Setting region to {} from {}", region, origin);
+  }
+
   /**
    * Given a endpoint string, create the endpoint URI.
    *
@@ -229,4 +305,23 @@ public class DefaultS3ClientFactory extends Configured
       throw new IllegalArgumentException(e);
     }
   }
+
+  /**
+   * Parses the endpoint to get the region.
+   * If endpoint is the central one, use US_EAST_1.
+   *
+   * @param endpoint the configure endpoint.
+   * @return the S3 region, null if unable to resolve from endpoint.
+   */
+  private static Region getS3RegionFromEndpoint(String endpoint) {
+
+    if(!endpoint.endsWith(CENTRAL_ENDPOINT)) {
+      LOG.debug("Endpoint {} is not the default; parsing", endpoint);
+      return AwsHostNameUtils.parseSigningRegion(endpoint, S3_SERVICE_NAME).orElse(null);
+    }
+
+    // endpoint is for US_EAST_1;
+    return Region.US_EAST_1;
+  }
+
 }

+ 1 - 84
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -34,7 +34,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.EnumSet;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
@@ -54,7 +53,6 @@ import javax.annotation.Nullable;
 
 import software.amazon.awssdk.core.ResponseInputStream;
 import software.amazon.awssdk.core.exception.SdkException;
-import software.amazon.awssdk.regions.Region;
 import software.amazon.awssdk.services.s3.S3AsyncClient;
 import software.amazon.awssdk.services.s3.S3Client;
 import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest;
@@ -83,7 +81,6 @@ import software.amazon.awssdk.services.s3.model.ObjectIdentifier;
 import software.amazon.awssdk.services.s3.model.PutObjectRequest;
 import software.amazon.awssdk.services.s3.model.PutObjectResponse;
 import software.amazon.awssdk.services.s3.model.S3Error;
-import software.amazon.awssdk.services.s3.model.S3Exception;
 import software.amazon.awssdk.services.s3.model.S3Object;
 import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest;
 import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler;
@@ -98,7 +95,6 @@ import software.amazon.awssdk.transfer.s3.model.CopyRequest;
 import software.amazon.awssdk.transfer.s3.model.FileUpload;
 import software.amazon.awssdk.transfer.s3.model.UploadFileRequest;
 
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.impl.prefetch.ExecutorServiceFuturePool;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -246,7 +242,6 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.ARN_BUCKET_OPTION;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DEFAULT_UPLOAD_PART_COUNT_LIMIT;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.DELETE_CONSIDERED_IDEMPOTENT;
-import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_301_MOVED_PERMANENTLY;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_403_FORBIDDEN;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404_NOT_FOUND;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.UPLOAD_PART_COUNT_LIMIT;
@@ -332,8 +327,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   private int executorCapacity;
   private long multiPartThreshold;
   public static final Logger LOG = LoggerFactory.getLogger(S3AFileSystem.class);
-  /** Exactly once log to warn about setting the region in config to avoid probe. */
-  private static final LogExactlyOnce SET_REGION_WARNING = new LogExactlyOnce(LOG);
 
   /** Log to warn of storage class configuration problems. */
   private static final LogExactlyOnce STORAGE_CLASS_WARNING = new LogExactlyOnce(LOG);
@@ -461,8 +454,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   private String scheme = FS_S3A;
 
-  private final static Map<String, Region> BUCKET_REGIONS = new HashMap<>();
-
   /** Add any deprecated keys. */
   @SuppressWarnings("deprecation")
   private static void addDeprecatedKeys() {
@@ -870,9 +861,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         STORE_EXISTS_PROBE, bucket, null, () ->
             invoker.retry("doesBucketExist", bucket, true, () -> {
               try {
-                if (BUCKET_REGIONS.containsKey(bucket)) {
-                  return true;
-                }
                 s3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build());
                 return true;
               } catch (AwsServiceException ex) {
@@ -982,8 +970,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         ? conf.getTrimmed(AWS_REGION)
         : accessPoint.getRegion();
 
-    Region region = getS3Region(configuredRegion);
-
     S3ClientFactory.S3ClientCreationParameters parameters =
         new S3ClientFactory.S3ClientCreationParameters()
         .withCredentialSet(credentials)
@@ -998,7 +984,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
         .withMultipartCopyEnabled(isMultipartCopyEnabled)
         .withMultipartThreshold(multiPartThreshold)
         .withTransferManagerExecutor(unboundedThreadPool)
-        .withRegion(region);
+        .withRegion(configuredRegion);
 
     S3ClientFactory clientFactory = ReflectionUtils.newInstance(s3ClientFactoryClass, conf);
     s3Client = clientFactory.createS3Client(getUri(), parameters);
@@ -1019,75 +1005,6 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     s3AsyncClient = clientFactory.createS3AsyncClient(getUri(), parameters);
   }
 
-  /**
-   * Get the bucket region.
-   *
-   * @param region AWS S3 Region set in the config. This property may not be set, in which case
-   *               ask S3 for the region.
-   * @return region of the bucket.
-   */
-  private Region getS3Region(String region) throws IOException {
-
-    if (!StringUtils.isBlank(region)) {
-      return Region.of(region);
-    }
-
-    Region cachedRegion = BUCKET_REGIONS.get(bucket);
-
-    if (cachedRegion != null) {
-      LOG.debug("Got region {} for bucket {} from cache", cachedRegion, bucket);
-      return cachedRegion;
-    }
-
-    Region s3Region = trackDurationAndSpan(STORE_REGION_PROBE, bucket, null,
-        () -> invoker.retry("getS3Region", bucket, true, () -> {
-          try {
-
-            SET_REGION_WARNING.warn(
-                "Getting region for bucket {} from S3, this will slow down FS initialisation. "
-                    + "To avoid this, set the region using property {}", bucket,
-                FS_S3A_BUCKET_PREFIX + bucket + ".endpoint.region");
-
-            // build a s3 client with region eu-west-1 that can be used to get the region of the
-            // bucket. Using eu-west-1, as headBucket() doesn't work with us-east-1. This is because
-            // us-east-1 uses the endpoint s3.amazonaws.com, which resolves bucket.s3.amazonaws.com
-            // to the actual region the bucket is in. As the request is signed with us-east-1 and
-            // not the bucket's region, it fails.
-            S3Client getRegionS3Client =
-                S3Client.builder().region(Region.EU_WEST_1).credentialsProvider(credentials)
-                    .build();
-
-            HeadBucketResponse headBucketResponse =
-                getRegionS3Client.headBucket(HeadBucketRequest.builder().bucket(bucket).build());
-
-            Region bucketRegion = Region.of(
-                headBucketResponse.sdkHttpResponse().headers().get(BUCKET_REGION_HEADER).get(0));
-            BUCKET_REGIONS.put(bucket, bucketRegion);
-
-            return bucketRegion;
-          } catch (S3Exception exception) {
-            if (exception.statusCode() == SC_301_MOVED_PERMANENTLY) {
-              Region bucketRegion = Region.of(
-                  exception.awsErrorDetails().sdkHttpResponse().headers().get(BUCKET_REGION_HEADER)
-                      .get(0));
-              BUCKET_REGIONS.put(bucket, bucketRegion);
-
-              return bucketRegion;
-            }
-
-            if (exception.statusCode() == SC_404_NOT_FOUND) {
-              throw new UnknownStoreException("s3a://" + bucket + "/",
-                  " Bucket does not exist: " + exception,
-                  exception);
-            }
-
-            throw exception;
-          }
-        }));
-
-    return s3Region;
-  }
-
   /**
    * Initialize and launch the audit manager and service.
    * As this takes the FS IOStatistics store, it must be invoked

+ 16 - 17
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java

@@ -28,7 +28,6 @@ import java.util.concurrent.Executor;
 
 import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
-import software.amazon.awssdk.regions.Region;
 import software.amazon.awssdk.services.s3.S3AsyncClient;
 import software.amazon.awssdk.services.s3.S3Client;
 import software.amazon.awssdk.transfer.s3.S3TransferManager;
@@ -169,7 +168,7 @@ public interface S3ClientFactory {
     /**
      * Region of the S3 bucket.
      */
-    private Region region;
+    private String region;
 
 
     /**
@@ -386,42 +385,42 @@ public interface S3ClientFactory {
     }
 
     /**
-     * Set region.
+     * Set the multipart flag..
      *
      * @param value new value
      * @return the builder
      */
-    public S3ClientCreationParameters withRegion(
-        final Region value) {
-      region = value;
+    public S3ClientCreationParameters withMultipartCopyEnabled(final boolean value) {
+      this.multipartCopy = value;
       return this;
     }
 
     /**
-     * Get the region.
-     * @return invoker
+     * Get the multipart flag.
+     * @return multipart flag
      */
-    public Region getRegion() {
-      return region;
+    public boolean isMultipartCopy() {
+      return multipartCopy;
     }
 
     /**
-     * Set the multipart flag..
+     * Set region.
      *
      * @param value new value
      * @return the builder
      */
-    public S3ClientCreationParameters withMultipartCopyEnabled(final boolean value) {
-      this.multipartCopy = value;
+    public S3ClientCreationParameters withRegion(
+        final String value) {
+      region = value;
       return this;
     }
 
     /**
-     * Get the multipart flag.
-     * @return multipart flag
+     * Get the region.
+     * @return invoker
      */
-    public boolean isMultipartCopy() {
-      return multipartCopy;
+    public String getRegion() {
+      return region;
     }
   }
 }

+ 0 - 5
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java

@@ -548,11 +548,6 @@ public enum Statistic {
       StoreStatisticNames.STORE_IO_THROTTLE_RATE,
       "Rate of S3 request throttling",
       TYPE_QUANTILE),
-  STORE_REGION_PROBE(
-      StoreStatisticNames.STORE_REGION_PROBE,
-      "Store Region Probe",
-      TYPE_DURATION
-  ),
 
   /*
    * Delegation Token Operations.

+ 7 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java

@@ -406,6 +406,13 @@ public abstract class S3GuardTool extends Configured implements Tool,
         // Note and continue.
         LOG.debug("failed to get bucket location", e);
         println(out, LOCATION_UNKNOWN);
+
+        // it may be the bucket is not found; we can't differentiate
+        // that and handle third party store issues where the API may
+        // not work.
+        // Fallback to looking for bucket root attributes.
+        println(out, "Probing for bucket existence");
+        fs.listXAttrs(new Path("/"));
       }
 
       // print any auth paths for directory marker info

+ 17 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.util.VersionInfo;
 import org.apache.http.HttpStatus;
 
 import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.EU_WEST_1;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
@@ -366,6 +367,7 @@ public class ITestS3AConfiguration {
       throws Exception {
 
     conf = new Configuration();
+    skipIfCrossRegionClient(conf);
     conf.set(Constants.PATH_STYLE_ACCESS, Boolean.toString(true));
     assertTrue(conf.getBoolean(Constants.PATH_STYLE_ACCESS, false));
 
@@ -404,6 +406,7 @@ public class ITestS3AConfiguration {
   @Test
   public void testDefaultUserAgent() throws Exception {
     conf = new Configuration();
+    skipIfCrossRegionClient(conf);
     fs = S3ATestUtils.createTestFileSystem(conf);
     assertNotNull(fs);
     S3Client s3 = getS3Client("User Agent");
@@ -417,6 +420,7 @@ public class ITestS3AConfiguration {
   @Test
   public void testCustomUserAgent() throws Exception {
     conf = new Configuration();
+    skipIfCrossRegionClient(conf);
     conf.set(Constants.USER_AGENT_PREFIX, "MyApp");
     fs = S3ATestUtils.createTestFileSystem(conf);
     assertNotNull(fs);
@@ -431,6 +435,7 @@ public class ITestS3AConfiguration {
   @Test
   public void testRequestTimeout() throws Exception {
     conf = new Configuration();
+    skipIfCrossRegionClient(conf);
     conf.set(REQUEST_TIMEOUT, "120");
     fs = S3ATestUtils.createTestFileSystem(conf);
     S3Client s3 = getS3Client("Request timeout (ms)");
@@ -610,4 +615,16 @@ public class ITestS3AConfiguration {
       return stsSignerCalled;
     }
   }
+
+  /**
+   * Skip a test if client created is cross region.
+   * @param configuration configuration to probe
+   */
+  private static void skipIfCrossRegionClient(
+      Configuration configuration) {
+    if (configuration.get(ENDPOINT, null) == null
+        && configuration.get(AWS_REGION, null) == null) {
+      skip("Skipping test as cross region client is in use ");
+    }
+  }
 }

+ 104 - 49
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java

@@ -27,6 +27,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.assertj.core.api.Assertions;
+import org.junit.Ignore;
 import org.junit.Test;
 import software.amazon.awssdk.awscore.AwsExecutionAttribute;
 import software.amazon.awssdk.awscore.exception.AwsServiceException;
@@ -42,8 +43,9 @@ import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
 import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
 import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
-import static org.apache.hadoop.fs.s3a.Statistic.STORE_REGION_PROBE;
 import static org.apache.hadoop.io.IOUtils.closeStream;
+import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT;
+
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
@@ -54,9 +56,17 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
 
   private static final String AWS_ENDPOINT_TEST = "test-endpoint";
 
-  private static final String USW_2_BUCKET = "landsat-pds";
+  private static final String US_EAST_1 = "us-east-1";
+
+  private static final String US_EAST_2 = "us-east-2";
+
+  private static final String US_WEST_2 = "us-west-2";
+
+  private static final String EU_WEST_2 = "eu-west-2";
 
-  public static final String USW_2_STORE = "s3a://" + USW_2_BUCKET;
+  private static final String CN_NORTHWEST_1 = "cn-northwest-1";
+
+  private static final String US_GOV_EAST_1 = "us-gov-east-1";
 
   /**
    * If anyone were ever to create a bucket with this UUID pair it would break the tests.
@@ -64,6 +74,14 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
   public static final String UNKNOWN_BUCKET = "23FA76D4-5F17-48B8-9D7D-9050269D0E40"
       + "-8281BAF2-DBCF-47AA-8A27-F2FA3589656A";
 
+  private static final String EU_WEST_2_ENDPOINT = "s3.eu-west-2.amazonaws.com";
+
+  private static final String CN_ENDPOINT = "s3.cn-northwest-1.amazonaws.com.cn";
+
+  private static final String GOV_ENDPOINT = "s3-fips.us-gov-east-1.amazonaws.com";
+
+  private static final String VPC_ENDPOINT = "vpce-1a2b3c4d-5e6f.s3.us-west-2.vpce.amazonaws.com";
+
   /**
    * New FS instance which will be closed in teardown.
    */
@@ -75,11 +93,6 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
     super.teardown();
   }
 
-  /**
-   * Test to verify that not setting the region config, will lead to the client factory making
-   * a HEAD bucket call to configure the correct region. If an incorrect region is set, the HEAD
-   * bucket call in this test will raise an exception.
-   */
   @Test
   public void testWithoutRegionConfig() throws IOException {
     describe("Verify that region lookup takes place");
@@ -96,7 +109,6 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
     } catch (UnknownHostException | UnknownStoreException | AccessDeniedException allowed) {
       // these are all valid failure modes from different test environments.
     }
-    assertRegionProbeCount(1);
   }
 
   @Test
@@ -115,82 +127,128 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
     } catch (UnknownHostException | UnknownStoreException expected) {
       // this is good.
     }
-    assertRegionProbeCount(1);
   }
 
+  @Test
+  public void testEndpointOverride() throws Throwable {
+    describe("Create a client with a configured endpoint");
+    Configuration conf = getConfiguration();
+
+    S3Client client = createS3Client(conf, AWS_ENDPOINT_TEST, null, US_EAST_2);
+
+    intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+        HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
+  }
 
   @Test
-  public void testWithRegionConfig() throws IOException, URISyntaxException {
-    describe("Verify that region lookup is skipped if the region property is set");
+  public void testCentralEndpoint() throws Throwable {
+    describe("Create a client with the central endpoint");
     Configuration conf = getConfiguration();
-    removeBaseAndBucketOverrides(conf, AWS_REGION, PATH_STYLE_ACCESS);
 
-    conf.set(AWS_REGION, "us-east-2");
+    S3Client client = createS3Client(conf, CENTRAL_ENDPOINT, null, US_EAST_1);
 
-    newFS = new S3AFileSystem();
-    newFS.initialize(new URI(USW_2_STORE), conf);
-    assertRegionProbeCount(0);
+    intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+        HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
   }
 
   @Test
-  public void testRegionCache() throws IOException, URISyntaxException {
-    describe("Verify that region lookup is cached on the second attempt");
+  public void testWithRegionConfig() throws Throwable {
+    describe("Create a client with a configured region");
     Configuration conf = getConfiguration();
-    removeBaseAndBucketOverrides(USW_2_BUCKET, conf, AWS_REGION, PATH_STYLE_ACCESS);
 
-    newFS = new S3AFileSystem();
+    S3Client client = createS3Client(conf, null, EU_WEST_2, EU_WEST_2);
 
-    newFS.initialize(new URI(USW_2_STORE), conf);
+    intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+        HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
+  }
 
-    assertRegionProbeCount(1);
-    closeStream(newFS);
 
-    // create a new instance
-    newFS = new S3AFileSystem();
-    newFS.initialize(new URI(USW_2_STORE), conf);
+  public void testEUWest2Endpoint() throws Throwable {
+    describe("Create a client with the eu west 2 endpoint");
+    Configuration conf = getConfiguration();
+
+    S3Client client = createS3Client(conf, EU_WEST_2_ENDPOINT, null, EU_WEST_2);
 
-    // value should already be cached.
-    assertRegionProbeCount(0);
+    intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+        HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
   }
 
-  private void assertRegionProbeCount(final int expected) {
-    Assertions.assertThat(newFS.getInstrumentation().getCounterValue(STORE_REGION_PROBE))
-        .describedAs("Incorrect number of calls made to get bucket region").isEqualTo(expected);
+  @Test
+  public void testWithRegionAndEndpointConfig() throws Throwable {
+    describe("Test that when both region and endpoint are configured, region takes precedence");
+    Configuration conf = getConfiguration();
+
+    S3Client client = createS3Client(conf, EU_WEST_2_ENDPOINT, US_WEST_2, US_WEST_2);
+
+    intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+        HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
   }
 
   @Test
-  public void testEndpointOverride() throws Throwable {
-    describe("Create a client with no region and the default endpoint");
+  public void testWithChinaEndpoint() throws Throwable {
+    describe("Test with a china endpoint");
     Configuration conf = getConfiguration();
 
-    S3Client client = createS3Client(conf, AWS_ENDPOINT_TEST);
+    S3Client client = createS3Client(conf, CN_ENDPOINT, null, CN_NORTHWEST_1);
 
     intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
         HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
   }
 
+  @Test
+  public void testWithGovCloudEndpoint() throws Throwable {
+    describe("Test with a gov cloud endpoint");
+    Configuration conf = getConfiguration();
+
+    S3Client client = createS3Client(conf, GOV_ENDPOINT, null, US_GOV_EAST_1);
+
+    intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+        HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
+  }
+
+  @Test
+  @Ignore("Pending HADOOP-18938. S3A region logic to handle vpce and non standard endpoints")
+  public void testWithVPCE() throws Throwable {
+    describe("Test with vpc endpoint");
+    Configuration conf = getConfiguration();
+
+    S3Client client = createS3Client(conf, VPC_ENDPOINT, null, US_WEST_2);
+
+    intercept(AwsServiceException.class, "Exception thrown by interceptor", () -> client.headBucket(
+        HeadBucketRequest.builder().bucket(getFileSystem().getBucket()).build()));
+  }
 
   class RegionInterceptor implements ExecutionInterceptor {
-    private boolean endpointOverridden;
+    private String endpoint;
+    private String region;
 
-    RegionInterceptor(boolean endpointOverridden) {
-      this.endpointOverridden = endpointOverridden;
+    RegionInterceptor(String endpoint, String region) {
+      this.endpoint = endpoint;
+      this.region = region;
     }
 
     @Override
     public void beforeExecution(Context.BeforeExecution context,
         ExecutionAttributes executionAttributes)  {
 
-      if (endpointOverridden) {
+      if (endpoint != null) {
         Assertions.assertThat(
                 executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN))
             .describedAs("Endpoint not overridden").isTrue();
 
         Assertions.assertThat(
                 executionAttributes.getAttribute(AwsExecutionAttribute.CLIENT_ENDPOINT).toString())
-            .describedAs("There is an endpoint mismatch").isEqualTo("https://" + AWS_ENDPOINT_TEST);
+            .describedAs("There is an endpoint mismatch").isEqualTo("https://" + endpoint);
+      } else {
+        Assertions.assertThat(
+                executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN))
+            .describedAs("Endpoint is overridden").isEqualTo(null);
       }
 
+      Assertions.assertThat(
+              executionAttributes.getAttribute(AwsExecutionAttribute.AWS_REGION).toString())
+          .describedAs("Incorrect region set").isEqualTo(region);
+
       // We don't actually want to make a request, so exit early.
       throw AwsServiceException.builder().message("Exception thrown by interceptor").build();
     }
@@ -202,23 +260,18 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
    * value.
    * @param conf configuration to use.
    * @param endpoint endpoint.
+   * @param expectedRegion the region that should be set in the client.
    * @return the client.
    * @throws URISyntaxException parse problems.
    * @throws IOException IO problems
    */
   @SuppressWarnings("deprecation")
   private S3Client createS3Client(Configuration conf,
-      String endpoint)
+      String endpoint, String configuredRegion, String expectedRegion)
       throws IOException {
 
-    boolean endpointOverridden = false;
-
-    if (endpoint != null && !endpoint.isEmpty()) {
-      endpointOverridden = true;
-    }
-
     List<ExecutionInterceptor> interceptors = new ArrayList<>();
-    interceptors.add(new RegionInterceptor(endpointOverridden));
+    interceptors.add(new RegionInterceptor(endpoint, expectedRegion));
 
     DefaultS3ClientFactory factory
         = new DefaultS3ClientFactory();
@@ -229,7 +282,9 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
         .withEndpoint(endpoint)
         .withMetrics(new EmptyS3AStatisticsContext()
             .newStatisticsFromAwsSdk())
-        .withExecutionInterceptors(interceptors);
+        .withExecutionInterceptors(interceptors)
+        .withRegion(configuredRegion);
+
 
     S3Client client = factory.createS3Client(
         getFileSystem().getUri(),

+ 2 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java

@@ -26,7 +26,6 @@ import java.io.UncheckedIOException;
 import java.net.URI;
 import java.nio.file.AccessDeniedException;
 
-import software.amazon.awssdk.regions.Region;
 import software.amazon.awssdk.services.s3.S3Client;
 import software.amazon.awssdk.services.s3.model.HeadBucketResponse;
 import org.junit.AfterClass;
@@ -596,8 +595,8 @@ public class ITestSessionDelegationInFilesystem extends AbstractDelegationIT {
         .withPathUri(new URI("s3a://localhost/"))
         .withMetrics(new EmptyS3AStatisticsContext()
             .newStatisticsFromAwsSdk())
-        .withUserAgentSuffix("ITestSessionDelegationInFilesystem")
-        .withRegion(Region.US_WEST_2);
+        .withUserAgentSuffix("ITestSessionDelegationInFilesystem");
+
     S3Client s3 = factory.createS3Client(landsat, parameters);
 
     return Invoker.once("HEAD", host,

+ 11 - 7
hadoop-tools/hadoop-aws/src/test/resources/core-site.xml

@@ -31,11 +31,10 @@
   </property>
 
   <!-- Per-bucket configurations: landsat-pds -->
-
   <property>
-    <name>fs.s3a.bucket.landsat-pds.endpoint</name>
-    <value>${central.endpoint}</value>
-    <description>The endpoint for s3a://landsat-pds URLs</description>
+    <name>fs.s3a.bucket.landsat-pds.endpoint.region</name>
+    <value>us-west-2</value>
+    <description>The region for s3a://landsat-pds</description>
   </property>
 
   <property>
@@ -58,10 +57,9 @@
   </property>
 
   <!-- Per-bucket configurations: usgs-landsat -->
-
   <property>
-    <name>fs.s3a.bucket.usgs-landsat.endpoint</name>
-    <value>${central.endpoint}</value>
+    <name>fs.s3a.bucket.usgs-landsat.endpoint.region</name>
+    <value>us-west-2</value>
   </property>
 
   <property>
@@ -82,6 +80,12 @@
     <value>false</value>
   </property>
 
+  <!-- Per-bucket configurations: osm-pds -->
+  <property>
+    <name>fs.s3a.bucket.osm-pds.endpoint.region</name>
+    <value>us-east-1</value>
+    <description>The region for s3a://osm-pds</description>
+  </property>
 
   <!--
    This is the default endpoint, which can be used to interact