, ClientT> RegionResolution.Resolution configureEndpointAndRegion(
+ BuilderT builder, S3ClientCreationParameters parameters, Configuration conf) throws IOException {
- // FIPs? Log it, then reject any attempt to set an endpoint
- final boolean fipsEnabled = parameters.isFipsEnabled();
- if (fipsEnabled) {
- LOG.debug("Enabling FIPS mode");
- }
- // always setting it guarantees the value is non-null,
- // which tests expect.
- builder.fipsEnabled(fipsEnabled);
-
- if (endpoint != null) {
- boolean endpointEndsWithCentral =
- endpointStr.endsWith(CENTRAL_ENDPOINT);
- checkArgument(!fipsEnabled || endpointEndsWithCentral, "%s : %s",
- ERROR_ENDPOINT_WITH_FIPS,
- endpoint);
-
- // No region was configured,
- // determine the region from the endpoint.
- if (region == null) {
- region = getS3RegionFromEndpoint(endpointStr,
- endpointEndsWithCentral);
- if (region != null) {
- origin = "endpoint";
- }
- }
+ final RegionResolution.Resolution resolution =
+ calculateRegion(parameters, conf);
+ LOG.debug("Region Resolution: {}", resolution);
- // No need to override endpoint with "s3.amazonaws.com".
- // Let the client take care of endpoint resolution. Overriding
- // the endpoint with "s3.amazonaws.com" causes 400 Bad Request
- // errors for non-existent buckets and objects.
- // ref: https://github.com/aws/aws-sdk-java-v2/issues/4846
- if (!endpointEndsWithCentral) {
- builder.endpointOverride(endpoint);
- LOG.debug("Setting endpoint to {}", endpoint);
- } else {
- origin = "central endpoint with cross region access";
- LOG.debug("Enabling cross region access for endpoint {}",
- endpointStr);
- }
- }
+ // always setting to true or false guarantees the value is non-null,
+ // which tests expect.
+ builder.fipsEnabled(resolution.isUseFips());
- 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.region(region);
- origin = "cross region access fallback";
- } else if (configuredRegion.isEmpty()) {
+ final RegionResolution.RegionResolutionMechanism mechanism = resolution.getMechanism();
+ if (Sdk == mechanism) {
+ // handing off all resolution to SDK.
// 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);
+ DEFAULT_REGION_CHAIN.info(SDK_REGION_CHAIN_IN_USE);
LOG.debug(SDK_REGION_CHAIN_IN_USE);
- origin = "SDK region chain";
- }
- boolean isCrossRegionAccessEnabled = conf.getBoolean(AWS_S3_CROSS_REGION_ACCESS_ENABLED,
- AWS_S3_CROSS_REGION_ACCESS_ENABLED_DEFAULT);
- // s3 cross region access
- if (isCrossRegionAccessEnabled) {
- builder.crossRegionAccessEnabled(true);
+ } else {
+
+ // a region has been determined from configuration,
+ // or it is falling back to central region.
+
+ final Region region = resolution.getRegion();
+ builder.region(requireNonNull(region));
+ // s3 cross region access
+ if (resolution.isCrossRegionAccessEnabled()) {
+ builder.crossRegionAccessEnabled(true);
+ }
+ final URI endpointUri = resolution.getEndpointUri();
+ if (endpointUri != null && !resolution.isUseCentralEndpoint()) {
+ LOG.debug("Setting endpoint to {}", endpointUri);
+ builder.endpointOverride(endpointUri);
+ }
}
- LOG.debug("Setting region to {} from {} with cross region access {}",
- region, origin, isCrossRegionAccessEnabled);
+ return resolution;
}
/**
* Given a endpoint string, create the endpoint URI.
- *
+ * Kept in as subclasses use it.
* @param endpoint possibly null endpoint.
* @param conf config to build the URI from.
* @return an endpoint uri
*/
protected static URI getS3Endpoint(String endpoint, final Configuration conf) {
-
boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS);
-
- String protocol = secureConnections ? "https" : "http";
-
- if (endpoint == null || endpoint.isEmpty()) {
- // don't set an endpoint if none is configured, instead let the SDK figure it out.
- return null;
- }
-
- if (!endpoint.contains("://")) {
- endpoint = String.format("%s://%s", protocol, endpoint);
- }
-
- try {
- return new URI(endpoint);
- } catch (URISyntaxException e) {
- throw new IllegalArgumentException(e);
- }
- }
-
- /**
- * Parses the endpoint to get the region.
- * If endpoint is the central one, use US_EAST_2.
- *
- * @param endpoint the configure endpoint.
- * @param endpointEndsWithCentral true if the endpoint is configured as central.
- * @return the S3 region, null if unable to resolve from endpoint.
- */
- @VisibleForTesting
- static Region getS3RegionFromEndpoint(final String endpoint,
- final boolean endpointEndsWithCentral) {
-
- if (!endpointEndsWithCentral) {
- // S3 VPC endpoint parsing
- Matcher matcher = VPC_ENDPOINT_PATTERN.matcher(endpoint);
- if (matcher.find()) {
- LOG.debug("Mapping to VPCE");
- LOG.debug("Endpoint {} is vpc endpoint; parsing region as {}", endpoint, matcher.group(1));
- return Region.of(matcher.group(1));
- }
-
- LOG.debug("Endpoint {} is not the default; parsing", endpoint);
- return AwsHostNameUtils.parseSigningRegion(endpoint, S3_SERVICE_NAME).orElse(null);
- }
-
- // Select default region here to enable cross-region access.
- // If both "fs.s3a.endpoint" and "fs.s3a.endpoint.region" are empty,
- // Spark sets "fs.s3a.endpoint" to "s3.amazonaws.com".
- // This applies to Spark versions with the changes of SPARK-35878.
- // ref:
- // https://github.com/apache/spark/blob/v3.5.0/core/
- // src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala#L528
- // If we do not allow cross region access, Spark would not be able to
- // access any bucket that is not present in the given region.
- // Hence, we should use default region us-east-2 to allow cross-region
- // access.
- return Region.of(AWS_S3_DEFAULT_REGION);
+ return RegionResolution.buildEndpointUri(endpoint, secureConnections);
}
private static , ClientT> void
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RegionResolution.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RegionResolution.java
new file mode 100644
index 0000000000000..b55d6004fb04e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RegionResolution.java
@@ -0,0 +1,535 @@
+/*
+ * 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.impl;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Locale;
+import java.util.Optional;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import javax.annotation.Nullable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.awscore.util.AwsHostNameUtils;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.regions.providers.InstanceProfileRegionProvider;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.Invoker;
+import org.apache.hadoop.fs.s3a.Retries;
+import org.apache.hadoop.fs.s3a.S3ClientFactory;
+
+import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CROSS_REGION_ACCESS_ENABLED;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CROSS_REGION_ACCESS_ENABLED_DEFAULT;
+import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT;
+import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_SECURE_CONNECTIONS;
+import static org.apache.hadoop.fs.s3a.Constants.EC2_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.EMPTY_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.FIPS_ENDPOINT;
+import static org.apache.hadoop.fs.s3a.Constants.SDK_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.SECURE_CONNECTIONS;
+import static org.apache.hadoop.util.Preconditions.checkArgument;
+import static software.amazon.awssdk.regions.Region.US_EAST_2;
+
+/**
+ * Region resolution.
+ * This is complicated and can be a source of support escalations.
+ *
The V1 SDK was happy to take an endpoint and
+ * work details out from there, possibly probing us-central-1 and cacheing
+ * the result.
+ *
The V2 SDK like the signing region and endpoint to be declared.
+ * The S3A connector has tried to mimic the V1 code, but lacks some features
+ * (use of environment variables, probing of EC2 IAM details) for which
+ * the SDK is better.
+ *
+ */
+public class RegionResolution {
+
+ protected static final Logger LOG =
+ LoggerFactory.getLogger(RegionResolution.class);
+
+ /**
+ * Service to ask SDK to parse.
+ */
+ private static final String S3_SERVICE_NAME = "s3";
+
+ /**
+ * Pattern to match vpce endpoints on.
+ */
+ private static final Pattern VPC_ENDPOINT_PATTERN =
+ Pattern.compile("^(?:.+\\.)?([a-z0-9-]+)\\.vpce\\.amazonaws\\.(?:com|com\\.cn)$");
+
+ /**
+ * Error message when an endpoint is set with FIPS enabled: {@value}.
+ */
+ @VisibleForTesting
+ public static final String ERROR_ENDPOINT_WITH_FIPS =
+ "Only S3 central endpoint cannot be set when " + FIPS_ENDPOINT + " is true";
+
+ /**
+ * Virtual hostnames MUST be used when using the FIPS endpoint.
+ */
+ public static final String FIPS_PATH_ACCESS_INCOMPATIBLE =
+ "Path style access must be disabled when " + FIPS_ENDPOINT + " is true";
+
+ /**
+ * String value for external region: {@value}.
+ */
+ public static final String EXTERNAL = "external";
+
+ /**
+ * External region, used for third party endpoints.
+ */
+ public static final Region EXTERNAL_REGION = Region.of(EXTERNAL);
+
+ /**
+ * How was the region resolved?
+ */
+ public enum RegionResolutionMechanism {
+
+ CalculatedFromEndpoint("Calculated from endpoint"),
+ ExternalEndpoint("External endpoint"),
+ FallbackToCentral("Fallback to central endpoint"),
+ ParseVpceEndpoint("Parse VPCE Endpoint"),
+ Ec2Metadata("EC2 Metadata"),
+ Sdk("SDK resolution chain"),
+ Specified("region specified");
+
+ /**
+ * Text of the mechanism.
+ */
+ private final String mechanism;
+
+ RegionResolutionMechanism(String mechanism) {
+ this.mechanism = mechanism;
+ }
+
+ /**
+ * String value of the resolution mechanism.
+ * @return the resolution mechanism.
+ */
+ public String getMechanism() {
+ return mechanism;
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder("RegionResolutionMechanism{");
+ sb.append("mechanism='").append(mechanism).append('\'');
+ sb.append('}');
+ return sb.toString();
+ }
+ }
+
+ /**
+ * The resolution of a region and endpoint..
+ */
+ public static final class Resolution {
+
+ /**
+ * Region: if null hand down to the SDK.
+ */
+ private Region region;
+
+ /**
+ * How was the region resolved?
+ * Null means unresolved.
+ */
+ private RegionResolutionMechanism mechanism;
+
+ /**
+ * Should FIPS be enabled?
+ */
+ private boolean useFips;
+
+ /**
+ * Should cross-region access be enabled?
+ */
+ private boolean crossRegionAccessEnabled;
+
+ /**
+ * Endpoint as string.
+ */
+ private String endpointStr;
+
+ /**
+ * Endpoint URI.
+ */
+ private URI endpointUri;
+
+ /**
+ * Use the central endpoint?
+ */
+ private boolean useCentralEndpoint;
+
+ public Resolution() {
+ }
+
+ /**
+ * Instantiate with a region and resolution mechanism.
+ * @param region region
+ * @param mechanism resolution mechanism.
+ */
+ public Resolution(final Region region, final RegionResolutionMechanism mechanism) {
+ this.region = region;
+ this.mechanism = mechanism;
+ }
+
+ /**
+ * Set the region.
+ * Declares the region as resolved even when the value is null (i.e. resolve to SDK).
+ * @param region region
+ * @param resolutionMechanism resolution mechanism.
+ * @return the builder
+ */
+ public Resolution withRegion(
+ @Nullable final Region region,
+ final RegionResolutionMechanism resolutionMechanism) {
+ this.region = region;
+ this.mechanism = requireNonNull(resolutionMechanism);
+ return this;
+ }
+
+ /**
+ * Set builder value.
+ * @param value new value
+ * @return the builder
+ */
+ public Resolution withUseFips(final boolean value) {
+ useFips = value;
+ return this;
+ }
+
+ /**
+ * Set builder value.
+ * @param value new value
+ * @return the builder
+ */
+ public Resolution withCrossRegionAccessEnabled(final boolean value) {
+ crossRegionAccessEnabled = value;
+ return this;
+ }
+
+ /**
+ * Set builder value.
+ * @param value new value
+ * @return the builder
+ */
+ public Resolution withEndpointStr(final String value) {
+ endpointStr = value;
+ return this;
+ }
+
+ public URI getEndpointUri() {
+ return endpointUri;
+ }
+
+ /**
+ * Set builder value.
+ * @param value new value
+ * @return the builder
+ */
+ public Resolution withEndpointUri(final URI value) {
+ endpointUri = value;
+ return this;
+ }
+
+ public Region getRegion() {
+ return region;
+ }
+
+ public boolean isUseFips() {
+ return useFips;
+ }
+
+ public boolean isCrossRegionAccessEnabled() {
+ return crossRegionAccessEnabled;
+ }
+
+ public RegionResolutionMechanism getMechanism() {
+ return mechanism;
+ }
+
+ public String getEndpointStr() {
+ return endpointStr;
+ }
+
+ public boolean isRegionResolved() {
+ return mechanism != null;
+ }
+
+ public boolean isUseCentralEndpoint() {
+ return useCentralEndpoint;
+ }
+
+ /**
+ * Set builder value.
+ * @param value new value
+ * @return the builder
+ */
+ public Resolution withUseCentralEndpoint(final boolean value) {
+ useCentralEndpoint = value;
+ return this;
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder("Resolution{");
+ sb.append("region=").append(region);
+ sb.append(", resolution=").append(mechanism);
+ sb.append(", useFips=").append(useFips);
+ sb.append(", crossRegionAccessEnabled=").append(crossRegionAccessEnabled);
+ sb.append(", endpointUri=").append(endpointUri);
+ sb.append(", useCentralEndpoint=").append(useCentralEndpoint);
+ sb.append('}');
+ return sb.toString();
+ }
+ }
+
+ /**
+ * Given a endpoint string, create the endpoint URI.
+ * @param endpoint possibly null endpoint.
+ * @param secureConnections use secure HTTPS connection?
+ * @return an endpoint uri or null if the endpoint was passed in was null/empty
+ * @throws IllegalArgumentException failure to parse the endpoint.
+ */
+ public static URI buildEndpointUri(String endpoint, final boolean secureConnections) {
+
+ String protocol = secureConnections ? "https" : "http";
+
+ if (endpoint == null || endpoint.isEmpty()) {
+ // don't set an endpoint if none is configured, instead let the SDK figure it out.
+ return null;
+ }
+
+ if (!endpoint.contains("://")) {
+ endpoint = String.format("%s://%s", protocol, endpoint);
+ }
+
+ try {
+ return new URI(endpoint);
+ } catch (URISyntaxException e) {
+ throw new IllegalArgumentException(e);
+ }
+ }
+
+ /**
+ * Parses the endpoint to get the region.
+ * If endpoint is the central one, use US_EAST_2.
+ * @param endpoint the configure endpoint.
+ * @param endpointEndsWithCentral true if the endpoint is configured as central.
+ * @return the S3 region resolution if possible from parsing the endpoint
+ */
+ @VisibleForTesting
+ public static Optional getS3RegionFromEndpoint(
+ final String endpoint,
+ final boolean endpointEndsWithCentral) {
+
+ if (!endpointEndsWithCentral) {
+ // S3 VPC endpoint parsing
+ Matcher matcher = VPC_ENDPOINT_PATTERN.matcher(endpoint);
+ if (matcher.find()) {
+ LOG.debug("Mapping to VPCE");
+ LOG.debug("Endpoint {} is VPC endpoint; parsing region as {}",
+ endpoint, matcher.group(1));
+ return Optional.of(new Resolution(
+ Region.of(matcher.group(1)),
+ RegionResolutionMechanism.ParseVpceEndpoint));
+ }
+
+ LOG.debug("Endpoint {} is not the default; parsing", endpoint);
+ return AwsHostNameUtils.parseSigningRegion(endpoint, S3_SERVICE_NAME)
+ .map(r ->
+ new Resolution(r, RegionResolutionMechanism.CalculatedFromEndpoint));
+ }
+
+ // No resolution.
+ return Optional.empty();
+ }
+
+ /**
+ * Is this an AWS endpoint, that is: has an endpoint been set which matches
+ * amazon.
+ * @param endpoint non-null endpoint URL
+ * @return true if this is amazonaws or amazonaws china
+ */
+ public static boolean isAwsEndpoint(final String endpoint) {
+ final String h = endpoint.toLowerCase(Locale.ROOT);
+ // Common AWS partitions: global (.amazonaws.com) and China (.amazonaws.com.cn).
+ return h.endsWith(".amazonaws.com")
+ || h.endsWith(".amazonaws.com.cn");
+ }
+
+
+ /**
+ * Does the region name refer to an SDK region?
+ * @param configuredRegion region in the configuration
+ * @return true if this is considered to refer to an SDK region.
+ */
+ public static boolean isSdkRegion(String configuredRegion) {
+ return SDK_REGION.equalsIgnoreCase(configuredRegion)
+ || EMPTY_REGION.equalsIgnoreCase(configuredRegion);
+ }
+
+ /**
+ * Does the region name refer to {@code "ec2"} in which case special handling
+ * is required.
+ * @param configuredRegion region in the configuration
+ * @return true if this is considered to refer to an SDK region.
+ */
+ public static boolean isEc2Region(String configuredRegion) {
+ return EC2_REGION.equalsIgnoreCase(configuredRegion);
+ }
+
+ /**
+ * Calculate the region and the final endpoint.
+ * @param parameters creation parameters
+ * @param conf configuration with other options.
+ * @return the resolved region and endpoint.
+ * @throws IOException if the client failed to communicate with the IAM service.
+ * @throws IllegalArgumentException failure to parse endpoint, or FIPS settings.
+ */
+ @Retries.OnceTranslated
+ public static Resolution calculateRegion(
+ final S3ClientFactory.S3ClientCreationParameters parameters,
+ final Configuration conf) throws IOException {
+
+ Resolution resolution = new Resolution();
+
+ // endpoint; may be null
+ final String endpointStr = parameters.getEndpoint();
+ boolean endpointDeclared = endpointStr != null && !endpointStr.isEmpty();
+ // will be null if endpointStr is null/empty
+ final URI endpoint = buildEndpointUri(endpointStr,
+ conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS));
+
+ final String configuredRegion = parameters.getRegion();
+
+ // If the region was configured, set it.
+ // this includes special handling of the sdk, ec2 and "" regions.
+ if (configuredRegion != null) {
+ checkArgument(!"null".equals(configuredRegion),
+ "null is region name");
+ if (isSdkRegion(configuredRegion)) {
+ resolution.withRegion(null, RegionResolutionMechanism.Sdk);
+ } else if (isEc2Region(configuredRegion)) {
+ // special EC2 handling
+ final Resolution r = getS3RegionFromEc2IAM();
+ resolution.withRegion(r.getRegion(), r.getMechanism());
+ } else {
+ resolution.withRegion(Region.of(configuredRegion),
+ RegionResolutionMechanism.Specified);
+ }
+ }
+
+ // central endpoint if no endpoint has been set, or it is explicitly
+ // requested
+ boolean endpointEndsWithCentral = !endpointDeclared
+ || endpointStr.endsWith(CENTRAL_ENDPOINT);
+
+ if (!resolution.isRegionResolved()) {
+ // parse from the endpoint and set if calculated
+ LOG.debug("Falling back to parsing region endpoint {}; endpointEndsWithCentral={}",
+ endpointStr, endpointEndsWithCentral);
+ final Optional regionFromEndpoint =
+ getS3RegionFromEndpoint(endpointStr, endpointEndsWithCentral);
+ if (regionFromEndpoint.isPresent()) {
+ regionFromEndpoint
+ .map(r ->
+ resolution.withRegion(r.getRegion(), r.getMechanism()));
+ }
+ }
+
+ // cross region setting.
+ resolution.withCrossRegionAccessEnabled(
+ conf.getBoolean(AWS_S3_CROSS_REGION_ACCESS_ENABLED,
+ AWS_S3_CROSS_REGION_ACCESS_ENABLED_DEFAULT));
+
+ // fips settings.
+ final boolean fipsEnabled = parameters.isFipsEnabled();
+ resolution.withUseFips(fipsEnabled);
+ if (fipsEnabled) {
+ // validate the FIPS settings
+ checkArgument(endpoint == null || endpointEndsWithCentral,
+ "%s : %s", ERROR_ENDPOINT_WITH_FIPS, endpoint);
+ checkArgument(!parameters.isPathStyleAccess(),
+ FIPS_PATH_ACCESS_INCOMPATIBLE);
+ }
+
+
+ if (!resolution.isRegionResolved()) {
+ // still not resolved.
+ if (!endpointDeclared || isAwsEndpoint(endpointStr)) {
+ // still failing to resolve the region
+ // fall back to central
+ resolution.withRegion(US_EAST_2, RegionResolutionMechanism.FallbackToCentral);
+ } else {
+ // we are not resolved and not an aws region.
+ // set the region to being "external"
+ resolution.withRegion(EXTERNAL_REGION, RegionResolutionMechanism.ExternalEndpoint);
+ }
+ }
+
+ // No need to override endpoint with "s3.amazonaws.com".
+ // Let the client take care of endpoint resolution. Overriding
+ // the endpoint with "s3.amazonaws.com" causes 400 Bad Request
+ // errors for non-existent buckets and objects.
+ // ref: https://github.com/aws/aws-sdk-java-v2/issues/4846
+ if (!endpointEndsWithCentral) {
+ LOG.debug("Setting endpoint to {}", endpoint);
+ resolution.withEndpointStr(endpointStr)
+ .withEndpointUri(endpoint)
+ .withUseCentralEndpoint(false);
+ } else {
+ resolution.withUseCentralEndpoint(true);
+ }
+
+ final Region r = resolution.getRegion();
+ if (r != null && Region.regions().contains(r)) {
+ // note that the region isn't known.
+ // not an issue for third party stores, otherwise it may be a region newer than
+ // that expected by the SDK. Hence: only log at debug.
+ LOG.debug("Region {} is not recognized by this SDK", r);
+ }
+ return resolution;
+ }
+
+ /**
+ * Probes EC2 Metadata for the region.
+ * This uses a class {@code InstanceProfileRegionProvider} which AWS
+ * declare as for internal use only.
+ * Linking/invocation should be caught and downgraded to returning an empty() option.
+ * @return the region from EC2 IAM.
+ * @throws IOException if the client failed to communicate with the IAM service.
+ */
+ @VisibleForTesting
+ @Retries.OnceTranslated
+ static Resolution getS3RegionFromEc2IAM() throws IOException {
+ return Invoker.once("Resolve EC2 Metadata", "/", () -> {
+ LOG.debug("Resolving region through EC2 Metadata");
+ final Region region = new InstanceProfileRegionProvider().getRegion();
+ return new Resolution(region, RegionResolutionMechanism.Ec2Metadata);
+ });
+ }
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
index 81a295345a8fc..31ab2668b558a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3AMockTest.java
@@ -27,6 +27,7 @@
import java.net.URI;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
@@ -35,7 +36,7 @@
/**
* Abstract base class for S3A unit tests using a mock S3 client.
*/
-public abstract class AbstractS3AMockTest {
+public abstract class AbstractS3AMockTest extends AbstractHadoopTestBase {
protected static final String BUCKET = "mock-bucket";
protected static final AwsServiceException NOT_FOUND =
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java
index 25efe7a06e5ae..7259526b2f88a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java
@@ -29,8 +29,10 @@
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
+import org.opentest4j.AssertionFailedError;
import software.amazon.awssdk.awscore.AwsExecutionAttribute;
import software.amazon.awssdk.awscore.exception.AwsServiceException;
+import software.amazon.awssdk.core.exception.SdkClientException;
import software.amazon.awssdk.core.interceptor.Context;
import software.amazon.awssdk.core.interceptor.ExecutionAttributes;
import software.amazon.awssdk.core.interceptor.ExecutionInterceptor;
@@ -54,11 +56,12 @@
import static org.apache.hadoop.fs.s3a.Constants.FIPS_ENDPOINT;
import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM;
-import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.ERROR_ENDPOINT_WITH_FIPS;
+import static org.apache.hadoop.fs.s3a.Constants.SDK_REGION;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeNotS3ExpressFileSystem;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeStoreAwsHosted;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.fs.s3a.impl.RegionResolution.ERROR_ENDPOINT_WITH_FIPS;
import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.DEFAULT_REQUESTER_PAYS_BUCKET_NAME;
import static org.apache.hadoop.io.IOUtils.closeStream;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -113,6 +116,9 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
* Text to include in assertions.
*/
private static final AtomicReference EXPECTED_MESSAGE = new AtomicReference<>();
+
+ public static final String INCORRECT_REGION_SET = "Incorrect region set";
+
/**
* New FS instance which will be closed in teardown.
*/
@@ -223,6 +229,32 @@ public void testWithRegionConfig() throws Throwable {
expectInterceptorException(client);
}
+ /**
+ * This hands off resolution to the SDK which may fail if nothing can be found
+ * (non-EC2; no AWS_REGION env var or through {@code ~/.aws/config}.
+ * There's separate handling for the different failure modes so this
+ * test will work in all deployments.
+ */
+ @Test
+ public void testWithSDKRegionConfig() throws Throwable {
+ describe("Create a client with an SDK region");
+ Configuration conf = getConfiguration();
+
+ try {
+ S3Client client = createS3Client(conf, CENTRAL_ENDPOINT, SDK_REGION, null, false);
+
+ expectInterceptorException(client);
+ } catch (SdkClientException e) {
+ Assertions.assertThat(e)
+ .describedAs("Exception raised due to unable to resolve region")
+ .hasMessageContaining("region");
+ } catch (AssertionFailedError e) {
+ Assertions.assertThat(e)
+ .describedAs("Exception raised region resolution working on local system")
+ .hasMessageContaining(INCORRECT_REGION_SET);
+ }
+ }
+
@Test
public void testWithFips() throws Throwable {
describe("Create a client with fips enabled");
@@ -646,7 +678,7 @@ public void beforeExecution(Context.BeforeExecution context,
}
Assertions.assertThat(reg)
- .describedAs("Incorrect region set in %s. Client Config=%s",
+ .describedAs(INCORRECT_REGION_SET + " in %s. Client Config=%s",
state, EXPECTED_MESSAGE.get())
.isEqualTo(region);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEndpointParsing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEndpointParsing.java
index 8be0708cad542..864cedeccd0b3 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEndpointParsing.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AEndpointParsing.java
@@ -18,10 +18,14 @@
package org.apache.hadoop.fs.s3a;
+import java.util.Optional;
+
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import software.amazon.awssdk.regions.Region;
+import org.apache.hadoop.fs.s3a.impl.RegionResolution;
+
public class TestS3AEndpointParsing extends AbstractS3AMockTest {
private static final String VPC_ENDPOINT = "vpce-1a2b3c4d-5e6f.s3.us-west-2.vpce.amazonaws.com";
@@ -31,13 +35,19 @@ public class TestS3AEndpointParsing extends AbstractS3AMockTest {
@Test
public void testVPCEndpoint() {
- Region region = DefaultS3ClientFactory.getS3RegionFromEndpoint(VPC_ENDPOINT, false);
- Assertions.assertThat(region).isEqualTo(Region.of(US_WEST_2));
+ Optional
+ region = RegionResolution.getS3RegionFromEndpoint(VPC_ENDPOINT, false);
+ Assertions.assertThat(region).get()
+ .extracting(RegionResolution.Resolution::getRegion)
+ .isEqualTo(Region.of(US_WEST_2));
}
@Test
public void testNonVPCEndpoint() {
- Region region = DefaultS3ClientFactory.getS3RegionFromEndpoint(NON_VPC_ENDPOINT, false);
- Assertions.assertThat(region).isEqualTo(Region.of(EU_WEST_1));
+ Optional
+ region = RegionResolution.getS3RegionFromEndpoint(NON_VPC_ENDPOINT, false);
+ Assertions.assertThat(region).get()
+ .extracting(RegionResolution.Resolution::getRegion)
+ .isEqualTo(Region.of(EU_WEST_1));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRegionResolution.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRegionResolution.java
new file mode 100644
index 0000000000000..015808e3b483c
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRegionResolution.java
@@ -0,0 +1,353 @@
+/*
+ * 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.impl;
+
+import java.io.IOException;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.regions.Region;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.S3ClientFactory;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT;
+import static org.apache.hadoop.fs.s3a.Constants.EC2_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.SDK_REGION;
+import static org.apache.hadoop.fs.s3a.impl.RegionResolution.ERROR_ENDPOINT_WITH_FIPS;
+import static org.apache.hadoop.fs.s3a.impl.RegionResolution.calculateRegion;
+import static org.apache.hadoop.fs.s3a.impl.RegionResolution.isEc2Region;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Test region resolution logic in {@link RegionResolution}.
+ * These are based on {@code ITestS3AEndpointRegion}.
+ */
+public class TestRegionResolution extends AbstractHadoopTestBase {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestRegionResolution.class);
+
+ 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";
+
+ private static final String CN_NORTHWEST_1 = "cn-northwest-1";
+
+ private static final String US_GOV_EAST_1 = "us-gov-east-1";
+
+ 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";
+
+ private static final String CN_VPC_ENDPOINT =
+ "vpce-1a2b3c4d-5e6f.s3.cn-northwest-1.vpce.amazonaws.com.cn";
+
+
+ private Configuration getConfiguration() {
+ return new Configuration(false);
+ }
+
+ /**
+ * Describe a test. This is a replacement for javadocs
+ * where the tests role is printed in the log output
+ * @param text description
+ */
+ protected void describe(String text) {
+ LOG.info(text);
+ }
+
+ private RegionResolution.Resolution resolve(Configuration conf,
+ String endpoint,
+ String configuredRegion,
+ boolean isFips,
+ String expectedRegion,
+ final RegionResolution.RegionResolutionMechanism expectedMechanism) throws IOException {
+ S3ClientFactory.S3ClientCreationParameters parameters =
+ new S3ClientFactory.S3ClientCreationParameters()
+ .withEndpoint(endpoint)
+ .withRegion(configuredRegion)
+ .withFipsEnabled(isFips);
+ final RegionResolution.Resolution resolved = calculateRegion(parameters, conf);
+
+ // check the region
+ if (expectedRegion != null) {
+ Assertions.assertThat(resolved.getRegion())
+ .describedAs("Resolved region %s", resolved)
+ .isNotNull()
+ .isEqualTo(Region.of(expectedRegion));
+ } else {
+ Assertions.assertThat(resolved.getRegion())
+ .describedAs("Resolved region %s", resolved)
+ .isNull();
+ }
+
+ // supplied resolution
+ if (expectedMechanism != null) {
+ assertMechanism(expectedMechanism, resolved);
+ }
+ return resolved;
+ }
+
+ /**
+ * Assert that a resolution used a specific mechanism.
+ * @param expectedMechanism expected mechanism.
+ * @param resolved resolved region
+ */
+ private static void assertMechanism(
+ final RegionResolution.RegionResolutionMechanism expectedMechanism,
+ final RegionResolution.Resolution resolved) {
+ Assertions.assertThat(resolved.getMechanism())
+ .describedAs("Resolution mechanism of %s", resolved)
+ .isEqualTo(expectedMechanism);
+ }
+
+ @Test
+ public void testWithVPCE() throws IOException {
+ resolve(getConfiguration(), VPC_ENDPOINT, null, false, US_WEST_2,
+ RegionResolution.RegionResolutionMechanism.ParseVpceEndpoint);
+ }
+
+ @Test
+ public void testWithChinaVPCE() throws IOException {
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), CN_VPC_ENDPOINT, null, false,
+ CN_NORTHWEST_1, RegionResolution.RegionResolutionMechanism.ParseVpceEndpoint);
+ assertEndpoint(r, CN_VPC_ENDPOINT);
+ assertUseCentralValue(r, false);
+ }
+
+ @Test
+ public void testCentralEndpointNoRegion() throws IOException {
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), CENTRAL_ENDPOINT, null, false,
+ US_EAST_2,
+ RegionResolution.RegionResolutionMechanism.FallbackToCentral);
+ assertEndpoint(r, null);
+ assertUseCentralValue(r, true);
+ }
+
+ @Test
+ public void testCentralEndpointWithRegion() throws IOException {
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), CENTRAL_ENDPOINT, US_WEST_2, false,
+ US_WEST_2, RegionResolution.RegionResolutionMechanism.Specified);
+ assertEndpoint(r, null);
+ assertUseCentralValue(r, true);
+ }
+
+ @Test
+ public void testConfiguredRegion() throws IOException {
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), null, EU_WEST_2, false,
+ EU_WEST_2, RegionResolution.RegionResolutionMechanism.Specified);
+ // this still uses the central endpoint.
+ assertEndpoint(r, null);
+ assertUseCentralValue(r, true);
+ }
+
+ @Test
+ public void testSDKRegion() throws IOException {
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), null, SDK_REGION, false,
+ null, RegionResolution.RegionResolutionMechanism.Sdk);
+ // SDK handles endpoint logic.
+ assertEndpoint(r, null);
+ assertUseCentralValue(r, true);
+ }
+
+ @Test
+ public void testSDKUpperCaseRegion() throws IOException {
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), null, "SDK", false,
+ null, RegionResolution.RegionResolutionMechanism.Sdk);
+ // SDK handles endpoint logic.
+ assertEndpoint(r, null);
+ assertUseCentralValue(r, true);
+ }
+
+ @Test
+ public void testEmptyStringRegion() throws IOException {
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), null, "", false,
+ null, RegionResolution.RegionResolutionMechanism.Sdk);
+ // SDK handles endpoint logic.
+ assertEndpoint(r, null);
+ assertUseCentralValue(r, true);
+ }
+
+ @Test
+ public void testWithFipsNoEndpoint() throws IOException {
+ describe("Create a client with fips enabled");
+
+ resolve(getConfiguration(),
+ null, EU_WEST_2, true,
+ EU_WEST_2, RegionResolution.RegionResolutionMechanism.Specified);
+ }
+
+ /**
+ * Attempting to create a client with fips enabled and an endpoint specified
+ * fails during client construction.
+ */
+ @Test
+ public void testWithFipsAndEndpoint() throws Exception {
+ describe("Create a client with fips and an endpoint");
+
+ intercept(IllegalArgumentException.class, ERROR_ENDPOINT_WITH_FIPS, () ->
+ resolve(getConfiguration(), US_WEST_2, null, true, US_EAST_1, null));
+ }
+
+ @Test
+ public void testWithRegionConfig() throws IOException {
+ describe("Create a client with a configured region");
+
+ resolve(getConfiguration(), null, EU_WEST_2, false,
+ EU_WEST_2, RegionResolution.RegionResolutionMechanism.Specified);
+ }
+
+ @Test
+ public void testEUWest2Endpoint() throws IOException {
+ describe("specifying an eu-west-2 endpoint selects that region");
+
+ resolve(getConfiguration(), EU_WEST_2_ENDPOINT, null, false,
+ EU_WEST_2, RegionResolution.RegionResolutionMechanism.CalculatedFromEndpoint);
+ }
+
+ @Test
+ public void testWithRegionAndEndpointConfig() throws IOException {
+ describe("Test that when both region and endpoint are configured, region takes precedence");
+
+ resolve(getConfiguration(), EU_WEST_2_ENDPOINT, US_WEST_2, false,
+ US_WEST_2, RegionResolution.RegionResolutionMechanism.Specified);
+ }
+
+ @Test
+ public void testWithChinaEndpoint() throws IOException {
+ describe("Test with a china endpoint");
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), CN_ENDPOINT, null, false,
+ CN_NORTHWEST_1,
+ RegionResolution.RegionResolutionMechanism.CalculatedFromEndpoint);
+ assertEndpoint(r, CN_ENDPOINT);
+ assertUseCentralValue(r, false);
+ }
+
+ @Test
+ public void testWithGovCloudEndpoint() throws IOException {
+ describe("Test with a gov cloud endpoint");
+ final RegionResolution.Resolution r =
+ resolve(getConfiguration(), GOV_ENDPOINT, null, false,
+ US_GOV_EAST_1,
+ RegionResolution.RegionResolutionMechanism.CalculatedFromEndpoint);
+ assertEndpoint(r, GOV_ENDPOINT);
+ assertUseCentralValue(r, false);
+ }
+
+ @Test
+ public void testNullIsForbidden() throws Throwable {
+ describe("The region null is forbidden as a red flag of configuration problems");
+ intercept(IllegalArgumentException.class, () ->
+ resolve(getConfiguration(), null, "null", false,
+ null, null));
+ }
+
+
+ /**
+ * This does attempt to talk to EC2 IAM but it doesn't need cloud credentials
+ * and will succeed whether the information came back or not.
+ * What it does do is validate the codepath.
+ */
+ @Test
+ public void testEC2Region() {
+ describe("Attempt to resolve region through EC2");
+
+ final Configuration conf = new Configuration(false);
+ S3ClientFactory.S3ClientCreationParameters parameters =
+ new S3ClientFactory.S3ClientCreationParameters()
+ .withRegion(EC2_REGION);
+ try {
+ final RegionResolution.Resolution resolved = calculateRegion(parameters, conf);
+ // here the process is under EC2 and a region was returned.
+ LOG.info("EC2 IAM resolved metadata: {}", resolved);
+ assertMechanism(RegionResolution.RegionResolutionMechanism.Ec2Metadata, resolved);
+ } catch (IOException e) {
+ // expected on anything except EC2
+ LOG.info("Expected failure when EC2 IAM is not present", e);
+ }
+ }
+
+ @Test
+ public void testEc2RegionCaseLogic() throws Throwable {
+ Assertions.assertThat(isEc2Region("ec2"))
+ .describedAs("lower case ec2").isTrue();
+ Assertions.assertThat(isEc2Region("EC2"))
+ .describedAs("upper case ec2").isTrue();
+ }
+
+ @Test
+ public void testGcsRegion() throws Throwable {
+ resolve(getConfiguration(), "https://storage.googleapis.com", null, false,
+ RegionResolution.EXTERNAL,
+ RegionResolution.RegionResolutionMechanism.ExternalEndpoint);
+ }
+
+ @Test
+ public void testLocalhostRegion() throws Throwable {
+ resolve(getConfiguration(), "127.0.0.1", null, false,
+ RegionResolution.EXTERNAL,
+ RegionResolution.RegionResolutionMechanism.ExternalEndpoint);
+ }
+
+ /**
+ * Assert that an endpoint matches the expected value.
+ * @param r resolution
+ * @param expected expected value.
+ */
+ private static void assertEndpoint(final RegionResolution.Resolution r,
+ final String expected) {
+ Assertions.assertThat(r.getEndpointStr())
+ .describedAs("Endpoint of %s", r)
+ .isEqualTo(expected);
+ }
+
+ /**
+ * assert that the resolution {@code isUseCentralEndpoint()} value
+ * matches that expected.
+ * @param r resolution
+ * @param expected expected value.
+ */
+ private static void assertUseCentralValue(final RegionResolution.Resolution r,
+ final boolean expected) {
+ Assertions.assertThat(r.isUseCentralEndpoint())
+ .describedAs("Endpoint of %s", r)
+ .isEqualTo(expected);
+ }
+
+}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestBucketTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestBucketTool.java
index bbe9d74824b7a..a6889c8014bb1 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestBucketTool.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestBucketTool.java
@@ -35,10 +35,13 @@
import org.apache.hadoop.util.ExitUtil;
import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeNotS3ExpressFileSystem;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeS3ExpressFileSystem;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeStoreAwsHosted;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.expectErrorCode;
+import static org.apache.hadoop.fs.s3a.impl.RegionResolution.isEc2Region;
+import static org.apache.hadoop.fs.s3a.impl.RegionResolution.isSdkRegion;
import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE;
import static org.apache.hadoop.fs.s3a.tools.BucketTool.CREATE;
import static org.apache.hadoop.fs.s3a.tools.BucketTool.NO_ZONE_SUPPLIED;
@@ -142,6 +145,9 @@ public void testRecreateTestBucketS3Express() throws Throwable {
public void testRecreateTestBucketNonS3Express() throws Throwable {
assumeNotS3ExpressFileSystem(fs);
assumeStoreAwsHosted(fs);
+ // fix a region if resolution is handed down to sdk
+ assume("Skipping as SDK region logic active",
+ !isSdkRegion(region) && !isEc2Region(region));
intercept(AWSBadRequestException.class, OWNED,
() -> bucketTool.exec("bucket", d(CREATE),
d(OPT_REGION), region,