diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index 75db90092d151..5f4d958b45233 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -1529,6 +1529,26 @@ private Constants() { */ public static final String AWS_S3_ACCESSPOINT_REQUIRED = "fs.s3a.accesspoint.required"; + /** + * Explicit request for the SDK region resolution. + * Value: {@code}. + */ + public static final String SDK_REGION = "sdk"; + + /** + * Declare as running in EC2. + * Currently hands off to the SDK for resolution; it may change in future. + * Value: {@code}. + */ + public static final String EC2_REGION = "ec2"; + + /** + * An empty region is the historic fall-through to the SDK. + * Value: "" + */ + public static final String EMPTY_REGION = ""; + + /** * Flag for create performance. * This can be set in the {code createFile()} builder. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 41e904ec9de1b..0be716059b6cb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -20,16 +20,10 @@ import java.io.IOException; import java.net.URI; -import java.net.URISyntaxException; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.hadoop.classification.VisibleForTesting; -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.checksums.RequestChecksumCalculation; import software.amazon.awssdk.core.checksums.ResponseChecksumValidation; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; @@ -57,17 +51,14 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.s3a.impl.AWSClientConfig; +import org.apache.hadoop.fs.s3a.impl.RegionResolution; 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 java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESS_GRANTS_ENABLED; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED; -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.AWS_S3_DEFAULT_REGION; -import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT; -import static org.apache.hadoop.fs.s3a.Constants.FIPS_ENDPOINT; import static org.apache.hadoop.fs.s3a.Constants.HTTP_SIGNER_CLASS_NAME; import static org.apache.hadoop.fs.s3a.Constants.HTTP_SIGNER_ENABLED; import static org.apache.hadoop.fs.s3a.Constants.HTTP_SIGNER_ENABLED_DEFAULT; @@ -77,7 +68,9 @@ import static org.apache.hadoop.fs.s3a.auth.SignerFactory.createHttpSigner; import static org.apache.hadoop.fs.s3a.impl.AWSHeaders.REQUESTER_PAYS_HEADER; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AUTH_SCHEME_AWS_SIGV_4; -import static org.apache.hadoop.util.Preconditions.checkArgument; +import static org.apache.hadoop.fs.s3a.impl.RegionResolution.RegionResolutionMechanism.Ec2Metadata; +import static org.apache.hadoop.fs.s3a.impl.RegionResolution.RegionResolutionMechanism.Sdk; +import static org.apache.hadoop.fs.s3a.impl.RegionResolution.calculateRegion; /** @@ -92,11 +85,6 @@ public class DefaultS3ClientFactory extends Configured private static final String REQUESTER_PAYS_HEADER_VALUE = "requester"; - private static final String S3_SERVICE_NAME = "s3"; - - private static final Pattern VPC_ENDPOINT_PATTERN = - Pattern.compile("^(?:.+\\.)?([a-z0-9-]+)\\.vpce\\.amazonaws\\.(?:com|com\\.cn)$"); - /** * Subclasses refer to this. */ @@ -106,26 +94,14 @@ public class DefaultS3ClientFactory extends Configured /** * A one-off warning of default region chains in use. */ - private static final LogExactlyOnce WARN_OF_DEFAULT_REGION_CHAIN = + private static final LogExactlyOnce DEFAULT_REGION_CHAIN = new LogExactlyOnce(LOG); /** - * Warning message printed when the SDK Region chain is in use. + * 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); - - /** - * Error message when an endpoint is set with FIPS enabled: {@value}. - */ - @VisibleForTesting - public static final String ERROR_ENDPOINT_WITH_FIPS = - "Non central endpoint cannot be set when " + FIPS_ENDPOINT + " is true"; + "S3A filesystem client is using the SDK region resolution chain."; /** * A one-off log stating whether S3 Access Grants are enabled. @@ -319,162 +295,63 @@ protected ClientOverrideConfiguration.Builder createClientOverrideConfiguration( *
  • S3 cross region is enabled by default irrespective of region or endpoint * is set or not.
  • * - * * @param builder S3 client builder. * @param parameters parameter object - * @param conf conf configuration object + * @param conf conf configuration object * @param S3 client builder type * @param S3 client type + * @return how the region was resolved. * @throws IllegalArgumentException if endpoint is set when FIPS is enabled. */ - private , ClientT> void configureEndpointAndRegion( - BuilderT builder, S3ClientCreationParameters parameters, Configuration conf) { - final String endpointStr = parameters.getEndpoint(); - final URI endpoint = getS3Endpoint(endpointStr, conf); - - final 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); - } + private , 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,