Prechádzať zdrojové kódy

HADOOP-19050. S3A: Support S3 Access Grants (#6544)

This adds support for Amazon S3 Access Grants to the S3A connector.

For more information, see:
* https://aws.amazon.com/s3/features/access-grants/
* https://github.com/aws/aws-s3-accessgrants-plugin-java-v2/

Contributed by Adnan Hemani
Adnan Hemani 1 rok pred
rodič
commit
8b2058a4e7

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

@@ -1624,4 +1624,21 @@ public final class Constants {
    * Value: {@value}.
    */
   public static final boolean DEFAULT_AWS_S3_CLASSLOADER_ISOLATION = true;
+
+  /**
+   * Flag {@value}
+   * to enable S3 Access Grants to control authorization to S3 data. More information:
+   * https://aws.amazon.com/s3/features/access-grants/
+   * and
+   * https://github.com/aws/aws-s3-accessgrants-plugin-java-v2/
+   */
+  public static final String AWS_S3_ACCESS_GRANTS_ENABLED = "fs.s3a.access.grants.enabled";
+
+  /**
+   * Flag {@value} to enable jobs fall back to the Job Execution IAM role in
+   * case they get Access Denied from the S3 Access Grants call. More information:
+   * https://github.com/aws/aws-s3-accessgrants-plugin-java-v2/
+   */
+  public static final String AWS_S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED =
+          "fs.s3a.access.grants.fallback.to.iam";
 }

+ 29 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java

@@ -37,6 +37,7 @@ import software.amazon.awssdk.http.auth.spi.scheme.AuthScheme;
 import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
 import software.amazon.awssdk.identity.spi.AwsCredentialsIdentity;
 import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.s3accessgrants.plugin.S3AccessGrantsPlugin;
 import software.amazon.awssdk.services.s3.S3AsyncClient;
 import software.amazon.awssdk.services.s3.S3BaseClientBuilder;
 import software.amazon.awssdk.services.s3.S3Client;
@@ -53,6 +54,8 @@ import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector;
 import org.apache.hadoop.fs.store.LogExactlyOnce;
 
 import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_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_DEFAULT_REGION;
 import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT;
 import static org.apache.hadoop.fs.s3a.Constants.FIPS_ENDPOINT;
@@ -112,6 +115,11 @@ public class DefaultS3ClientFactory extends Configured
   public static final String ERROR_ENDPOINT_WITH_FIPS =
       "Non central endpoint cannot be set when " + FIPS_ENDPOINT + " is true";
 
+  /**
+   * A one-off log stating whether S3 Access Grants are enabled.
+   */
+  private static final LogExactlyOnce LOG_S3AG_ENABLED = new LogExactlyOnce(LOG);
+
   @Override
   public S3Client createS3Client(
       final URI uri,
@@ -178,6 +186,8 @@ public class DefaultS3ClientFactory extends Configured
 
     configureEndpointAndRegion(builder, parameters, conf);
 
+    maybeApplyS3AccessGrantsConfigurations(builder, conf);
+
     S3Configuration serviceConfiguration = S3Configuration.builder()
         .pathStyleAccessEnabled(parameters.isPathStyleAccess())
         .checksumValidationEnabled(parameters.isChecksumValidationEnabled())
@@ -402,4 +412,23 @@ public class DefaultS3ClientFactory extends Configured
     return Region.of(AWS_S3_DEFAULT_REGION);
   }
 
+  private static <BuilderT extends S3BaseClientBuilder<BuilderT, ClientT>, ClientT> void
+  maybeApplyS3AccessGrantsConfigurations(BuilderT builder, Configuration conf) {
+    boolean isS3AccessGrantsEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false);
+    if (!isS3AccessGrantsEnabled){
+      LOG.debug("S3 Access Grants plugin is not enabled.");
+      return;
+    }
+
+    boolean isFallbackEnabled =
+        conf.getBoolean(AWS_S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED, false);
+    S3AccessGrantsPlugin accessGrantsPlugin =
+        S3AccessGrantsPlugin.builder()
+            .enableFallback(isFallbackEnabled)
+            .build();
+    builder.addPlugin(accessGrantsPlugin);
+    LOG_S3AG_ENABLED.info(
+        "S3 Access Grants plugin is enabled with IAM fallback set to {}", isFallbackEnabled);
+  }
+
 }

+ 10 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -494,6 +494,11 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
    */
   private String configuredRegion;
 
+  /**
+   * Are S3 Access Grants Enabled?
+   */
+  private boolean s3AccessGrantsEnabled;
+
   /** Add any deprecated keys. */
   @SuppressWarnings("deprecation")
   private static void addDeprecatedKeys() {
@@ -747,6 +752,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
       optimizedCopyFromLocal = conf.getBoolean(OPTIMIZED_COPY_FROM_LOCAL,
           OPTIMIZED_COPY_FROM_LOCAL_DEFAULT);
       LOG.debug("Using optimized copyFromLocal implementation: {}", optimizedCopyFromLocal);
+      s3AccessGrantsEnabled = conf.getBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, false);
     } catch (SdkException e) {
       // amazon client exception: stop all services then throw the translation
       cleanupWithLogger(LOG, span);
@@ -5516,6 +5522,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     case FIPS_ENDPOINT:
       return fipsEnabled;
 
+    // is S3 Access Grants enabled
+    case AWS_S3_ACCESS_GRANTS_ENABLED:
+      return s3AccessGrantsEnabled;
+
     default:
       return super.hasPathCapability(p, cap);
     }

+ 2 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java

@@ -36,6 +36,7 @@ import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_AVAILABLE;
 import static org.apache.hadoop.fs.CommonPathCapabilities.FS_CHECKSUMS;
 import static org.apache.hadoop.fs.CommonPathCapabilities.FS_MULTIPART_UPLOADER;
 import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS;
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESS_GRANTS_ENABLED;
 import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_OPERATIONS_PURGE_UPLOADS;
 import static org.apache.hadoop.fs.s3a.Constants.ENABLE_MULTI_DELETE;
 import static org.apache.hadoop.fs.s3a.Constants.FIPS_ENDPOINT;
@@ -272,6 +273,7 @@ public final class InternalConstants {
           FS_MULTIPART_UPLOADER,
           DIRECTORY_LISTING_INCONSISTENT,
           FIPS_ENDPOINT,
+          AWS_S3_ACCESS_GRANTS_ENABLED,
 
           // s3 specific
           STORE_CAPABILITY_AWS_V2,

+ 433 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/authentication.md

@@ -0,0 +1,433 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+# Authenticating with S3
+
+<!-- MACRO{toc|fromDepth=0|toDepth=2} -->
+
+Except when interacting with public S3 buckets, the S3A client
+needs the credentials needed to interact with buckets.
+
+The client supports multiple authentication mechanisms and can be configured as to
+which mechanisms to use, and their order of use. Custom implementations
+of `com.amazonaws.auth.AWSCredentialsProvider` may also be used.
+However, with the upgrade to AWS Java SDK V2 in Hadoop 3.4.0, these classes will need to be
+updated to implement `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
+For more information see [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html).
+
+### Authentication properties
+
+```xml
+<property>
+  <name>fs.s3a.access.key</name>
+  <description>AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
+</property>
+
+<property>
+  <name>fs.s3a.secret.key</name>
+  <description>AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
+</property>
+
+<property>
+  <name>fs.s3a.session.token</name>
+  <description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
+    as one of the providers.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>
+    org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
+    org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
+    software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider,
+    org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
+  </value>
+  <description>
+    Comma-separated class names of credential provider classes which implement
+    software.amazon.awssdk.auth.credentials.AwsCredentialsProvider.
+
+    When S3A delegation tokens are not enabled, this list will be used
+    to directly authenticate with S3 and other AWS services.
+    When S3A Delegation tokens are enabled, depending upon the delegation
+    token binding it may be used
+    to communicate wih the STS endpoint to request session/role
+    credentials.
+  </description>
+</property>
+
+<property>
+  <name>fs.s3a.aws.credentials.provider.mapping</name>
+  <description>
+    Comma-separated key-value pairs of mapped credential providers that are
+    separated by equal operator (=). The key can be used by
+    fs.s3a.aws.credentials.provider config, and it will be translated into
+    the specified value of credential provider class based on the key-value
+    pair provided by this config.
+
+    Example:
+    com.amazonaws.auth.AnonymousAWSCredentials=org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider,
+    com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider,
+    com.amazonaws.auth.InstanceProfileCredentialsProvider=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
+
+    With the above key-value pairs, if fs.s3a.aws.credentials.provider specifies
+    com.amazonaws.auth.AnonymousAWSCredentials, it will be remapped to
+    org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider by S3A while
+    preparing AWS credential provider list for any S3 access.
+    We can use the same credentials provider list for both v1 and v2 SDK clients.
+  </description>
+</property>
+```
+
+### <a name="auth_env_vars"></a> Authenticating via the AWS Environment Variables
+
+S3A supports configuration via [the standard AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment).
+
+The core environment variables are for the access key and associated secret:
+
+```bash
+export AWS_ACCESS_KEY_ID=my.aws.key
+export AWS_SECRET_ACCESS_KEY=my.secret.key
+```
+
+If the environment variable `AWS_SESSION_TOKEN` is set, session authentication
+using "Temporary Security Credentials" is enabled; the Key ID and secret key
+must be set to the credentials for that specific session.
+
+```bash
+export AWS_SESSION_TOKEN=SECRET-SESSION-TOKEN
+export AWS_ACCESS_KEY_ID=SESSION-ACCESS-KEY
+export AWS_SECRET_ACCESS_KEY=SESSION-SECRET-KEY
+```
+
+These environment variables can be used to set the authentication credentials
+instead of properties in the Hadoop configuration.
+
+*Important:*
+These environment variables are generally not propagated from client to server when
+YARN applications are launched. That is: having the AWS environment variables
+set when an application is launched will not permit the launched application
+to access S3 resources. The environment variables must (somehow) be set
+on the hosts/processes where the work is executed.
+
+### <a name="auth_providers"></a> Changing Authentication Providers
+
+The standard way to authenticate is with an access key and secret key set in
+the Hadoop configuration files.
+
+By default, the S3A client follows the following authentication chain:
+
+1. The options `fs.s3a.access.key`, `fs.s3a.secret.key` and `fs.s3a.sesson.key`
+   are looked for in the Hadoop XML configuration/Hadoop credential providers,
+   returning a set of session credentials if all three are defined.
+1. The `fs.s3a.access.key` and `fs.s3a.secret.key` are looked for in the Hadoop
+   XML configuration//Hadoop credential providers, returning a set of long-lived
+   credentials if they are defined.
+1. The [AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment),
+   are then looked for: these will return session or full credentials depending
+   on which values are set.
+1. An attempt is made to query the Amazon EC2 Instance/k8s container Metadata Service to
+   retrieve credentials published to EC2 VMs.
+
+S3A can be configured to obtain client authentication providers from classes
+which integrate with the AWS SDK by implementing the
+`software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`
+interface.
+This is done by listing the implementation classes, in order of
+preference, in the configuration option `fs.s3a.aws.credentials.provider`.
+In previous hadoop releases, providers were required to
+implement the AWS V1 SDK interface `com.amazonaws.auth.AWSCredentialsProvider`.
+Consult the [Upgrading S3A to AWS SDK V2](./aws_sdk_upgrade.html) documentation
+to see how to migrate credential providers.
+
+*Important*: AWS Credential Providers are distinct from _Hadoop Credential Providers_.
+As will be covered later, Hadoop Credential Providers allow passwords and other secrets
+to be stored and transferred more securely than in XML configuration files.
+AWS Credential Providers are classes which can be used by the Amazon AWS SDK to
+obtain an AWS login from a different source in the system, including environment
+variables, JVM properties and configuration files.
+
+All Hadoop `fs.s3a.` options used to store login details can all be secured
+in [Hadoop credential providers](../../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html);
+this is advised as a more secure way to store valuable secrets.
+
+There are a number of AWS Credential Providers inside the `hadoop-aws` JAR:
+
+| Hadoop module credential provider                              | Authentication Mechanism                         |
+|----------------------------------------------------------------|--------------------------------------------------|
+| `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider`     | Session Credentials in configuration             |
+| `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider`        | Simple name/secret credentials in configuration  |
+| `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider`     | Anonymous Login                                  |
+| `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider`  | [Assumed Role credentials](./assumed_roles.html) |
+| `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials                     |
+
+
+There are also many in the Amazon SDKs, with the common ones being as follows
+
+| classname                                                                        | description                  |
+|----------------------------------------------------------------------------------|------------------------------|
+| `software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` | AWS Environment Variables    |
+| `software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider`     | EC2 Metadata Credentials     |
+| `software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider`           | EC2/k8s Metadata Credentials |
+
+
+
+### <a name="auth_iam"></a> EC2 IAM Metadata Authentication with `InstanceProfileCredentialsProvider`
+
+Applications running in EC2 may associate an IAM role with the VM and query the
+[EC2 Instance Metadata Service](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-instance-metadata.html)
+for credentials to access S3.  Within the AWS SDK, this functionality is
+provided by `InstanceProfileCredentialsProvider`, which internally enforces a
+singleton instance in order to prevent throttling problem.
+
+### <a name="auth_named_profile"></a> Using Named Profile Credentials with `ProfileCredentialsProvider`
+
+You can configure Hadoop to authenticate to AWS using a [named profile](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-profiles.html).
+
+To authenticate with a named profile:
+
+1. Declare `software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider` as the provider.
+1. Set your profile via the `AWS_PROFILE` environment variable.
+1. Due to a [bug in version 1 of the AWS Java SDK](https://github.com/aws/aws-sdk-java/issues/803),
+   you'll need to remove the `profile` prefix from the AWS configuration section heading.
+
+   Here's an example of what your AWS configuration files should look like:
+
+    ```
+    $ cat ~/.aws/config
+    [user1]
+    region = us-east-1
+    $ cat ~/.aws/credentials
+    [user1]
+    aws_access_key_id = ...
+    aws_secret_access_key = ...
+    aws_session_token = ...
+    aws_security_token = ...
+    ```
+Note:
+
+1. The `region` setting is only used if `fs.s3a.endpoint.region` is set to the empty string.
+1. For the credentials to be available to applications running in a Hadoop cluster, the
+   configuration files MUST be in the `~/.aws/` directory on the local filesystem in
+   all hosts in the cluster.
+
+### <a name="auth_session"></a> Using Session Credentials with `TemporaryAWSCredentialsProvider`
+
+[Temporary Security Credentials](http://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp.html)
+can be obtained from the Amazon Security Token Service; these
+consist of an access key, a secret key, and a session token.
+
+To authenticate with these:
+
+1. Declare `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider` as the
+   provider.
+1. Set the session key in the property `fs.s3a.session.token`,
+   and the access and secret key properties to those of this temporary session.
+
+Example:
+
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider</value>
+</property>
+
+<property>
+  <name>fs.s3a.access.key</name>
+  <value>SESSION-ACCESS-KEY</value>
+</property>
+
+<property>
+  <name>fs.s3a.secret.key</name>
+  <value>SESSION-SECRET-KEY</value>
+</property>
+
+<property>
+  <name>fs.s3a.session.token</name>
+  <value>SECRET-SESSION-TOKEN</value>
+</property>
+```
+
+The lifetime of session credentials are fixed when the credentials
+are issued; once they expire the application will no longer be able to
+authenticate to AWS.
+
+### <a name="auth_anon"></a> Anonymous Login with `AnonymousAWSCredentialsProvider`
+
+Specifying `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` allows
+anonymous access to a publicly accessible S3 bucket without any credentials.
+It can be useful for accessing public data sets without requiring AWS credentials.
+
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider</value>
+</property>
+```
+
+Once this is done, there's no need to supply any credentials
+in the Hadoop configuration or via environment variables.
+
+This option can be used to verify that an object store does
+not permit unauthenticated access: that is, if an attempt to list
+a bucket is made using the anonymous credentials, it should fail —unless
+explicitly opened up for broader access.
+
+```bash
+hadoop fs -ls \
+ -D fs.s3a.aws.credentials.provider=org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider \
+ s3a://noaa-isd-pds/
+```
+
+1. Allowing anonymous access to an S3 bucket compromises
+   security and therefore is unsuitable for most use cases.
+
+1. If a list of credential providers is given in `fs.s3a.aws.credentials.provider`,
+   then the Anonymous Credential provider *must* come last. If not, credential
+   providers listed after it will be ignored.
+
+### <a name="auth_simple"></a> Simple name/secret credentials with `SimpleAWSCredentialsProvider`*
+
+This is the standard credential provider, which supports the secret
+key in `fs.s3a.access.key` and token in `fs.s3a.secret.key`
+values.
+
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
+</property>
+```
+
+This is the basic authenticator used in the default authentication chain.
+
+This means that the default S3A authentication chain can be defined as
+
+```xml
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>
+    org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
+    org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
+    software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider
+    org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
+  </value>
+</property>
+```
+
+## <a name="auth_security"></a> Protecting the AWS Credentials
+
+It is critical that you never share or leak your AWS credentials.
+Loss of credentials can leak/lose all your data, run up large bills,
+and significantly damage your organisation.
+
+1. Never share your secrets.
+
+1. Never commit your secrets into an SCM repository.
+   The [git secrets](https://github.com/awslabs/git-secrets) can help here.
+
+1. Never include AWS credentials in bug reports, files attached to them,
+   or similar.
+
+1. If you use the `AWS_` environment variables,  your list of environment variables
+   is equally sensitive.
+
+1. Never use root credentials.
+   Use IAM user accounts, with each user/application having its own set of credentials.
+
+1. Use IAM permissions to restrict the permissions individual users and applications
+   have. This is best done through roles, rather than configuring individual users.
+
+1. Avoid passing in secrets to Hadoop applications/commands on the command line.
+   The command line of any launched program is visible to all users on a Unix system
+   (via `ps`), and preserved in command histories.
+
+1. Explore using [IAM Assumed Roles](assumed_roles.html) for role-based permissions
+   management: a specific S3A connection can be made with a different assumed role
+   and permissions from the primary user account.
+
+1. Consider a workflow in which users and applications are issued with short-lived
+   session credentials, configuring S3A to use these through
+   the `TemporaryAWSCredentialsProvider`.
+
+1. Have a secure process in place for cancelling and re-issuing credentials for
+   users and applications. Test it regularly by using it to refresh credentials.
+
+1. In installations where Kerberos is enabled, [S3A Delegation Tokens](delegation_tokens.html)
+   can be used to acquire short-lived session/role credentials and then pass them
+   into the shared application. This can ensure that the long-lived secrets stay
+   on the local system.
+
+When running in EC2, the IAM EC2 instance credential provider will automatically
+obtain the credentials needed to access AWS services in the role the EC2 VM
+was deployed as.
+This AWS credential provider is enabled in S3A by default.
+
+## Custom AWS Credential Providers and Apache Spark
+
+Apache Spark employs two class loaders, one that loads "distribution" (Spark + Hadoop) classes and one that
+loads custom user classes. If the user wants to load custom implementations of AWS credential providers,
+custom signers, delegation token providers or any other dynamically loaded extension class
+through user provided jars she will need to set the following configuration:
+
+```xml
+<property>
+  <name>fs.s3a.classloader.isolation</name>
+  <value>false</value>
+</property>
+<property>
+  <name>fs.s3a.aws.credentials.provider</name>
+  <value>CustomCredentialsProvider</value>
+</property>
+```
+
+If the following property is not set or set to `true`, the following exception will be thrown:
+
+```
+java.io.IOException: From option fs.s3a.aws.credentials.provider java.lang.ClassNotFoundException: Class CustomCredentialsProvider not found
+```
+
+## <a name="auth_s3_access_grants"></a> S3 Authorization Using S3 Access Grants
+
+[S3 Access Grants](https://aws.amazon.com/s3/features/access-grants/) can be used to grant accesses to S3 data using IAM Principals.
+In order to enable S3 Access Grants, S3A utilizes the
+[S3 Access Grants plugin](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2) on all S3 clients,
+which is found within the AWS Java SDK bundle (v2.23.19+).
+
+S3A supports both cross-region access (by default) and the
+[fallback-to-IAM configuration](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2?tab=readme-ov-file#using-the-plugin)
+which allows S3A to fallback to using the IAM role (and its permission sets directly) to access S3 data in the case that S3 Access Grants
+is unable to authorize the S3 call.
+
+The following is how to enable this feature:
+
+```xml
+<property>
+  <name>fs.s3a.s3accessgrants.enabled</name>
+  <value>true</value>
+</property>
+<property>
+  <!--Optional: Defaults to False-->
+  <name>fs.s3a.s3accessgrants.fallback.to.iam</name>
+  <value>true</value>
+</property>
+```
+
+Note:
+1. S3A only enables the [S3 Access Grants plugin](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2) on the S3 clients
+as part of this feature. Any usage issues or bug reporting should be done directly at the plugin's
+[GitHub repo](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2/issues).
+
+For more details on using S3 Access Grants, please refer to
+[Managing access with S3 Access Grants](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-grants.html).

+ 1 - 382
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md

@@ -232,388 +232,7 @@ Also, please check [S3 endpoint and region settings in detail](connecting.html#s
 
 ## <a name="authenticating"></a> Authenticating with S3
 
-Except when interacting with public S3 buckets, the S3A client
-needs the credentials needed to interact with buckets.
-
-The client supports multiple authentication mechanisms and can be configured as to
-which mechanisms to use, and their order of use. Custom implementations
-of `com.amazonaws.auth.AWSCredentialsProvider` may also be used.
-However, with the upcoming upgrade to AWS Java SDK V2, these classes will need to be
-updated to implement `software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`.
-For more information see [Upcoming upgrade to AWS Java SDK V2](./aws_sdk_upgrade.html).
-
-### Authentication properties
-
-```xml
-<property>
-  <name>fs.s3a.access.key</name>
-  <description>AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.secret.key</name>
-  <description>AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.session.token</name>
-  <description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
-    as one of the providers.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <value>
-    org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
-    org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
-    software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider,
-    org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
-  </value>
-  <description>
-    Comma-separated class names of credential provider classes which implement
-    software.amazon.awssdk.auth.credentials.AwsCredentialsProvider.
-
-    When S3A delegation tokens are not enabled, this list will be used
-    to directly authenticate with S3 and other AWS services.
-    When S3A Delegation tokens are enabled, depending upon the delegation
-    token binding it may be used
-    to communicate wih the STS endpoint to request session/role
-    credentials.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.aws.credentials.provider.mapping</name>
-  <description>
-    Comma-separated key-value pairs of mapped credential providers that are
-    separated by equal operator (=). The key can be used by
-    fs.s3a.aws.credentials.provider config, and it will be translated into
-    the specified value of credential provider class based on the key-value
-    pair provided by this config.
-
-    Example:
-    com.amazonaws.auth.AnonymousAWSCredentials=org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider,
-    com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider,
-    com.amazonaws.auth.InstanceProfileCredentialsProvider=org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
-
-    With the above key-value pairs, if fs.s3a.aws.credentials.provider specifies
-    com.amazonaws.auth.AnonymousAWSCredentials, it will be remapped to
-    org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider by S3A while
-    preparing AWS credential provider list for any S3 access.
-    We can use the same credentials provider list for both v1 and v2 SDK clients.
-  </description>
-</property>
-```
-
-### <a name="auth_env_vars"></a> Authenticating via the AWS Environment Variables
-
-S3A supports configuration via [the standard AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment).
-
-The core environment variables are for the access key and associated secret:
-
-```bash
-export AWS_ACCESS_KEY_ID=my.aws.key
-export AWS_SECRET_ACCESS_KEY=my.secret.key
-```
-
-If the environment variable `AWS_SESSION_TOKEN` is set, session authentication
-using "Temporary Security Credentials" is enabled; the Key ID and secret key
-must be set to the credentials for that specific session.
-
-```bash
-export AWS_SESSION_TOKEN=SECRET-SESSION-TOKEN
-export AWS_ACCESS_KEY_ID=SESSION-ACCESS-KEY
-export AWS_SECRET_ACCESS_KEY=SESSION-SECRET-KEY
-```
-
-These environment variables can be used to set the authentication credentials
-instead of properties in the Hadoop configuration.
-
-*Important:*
-These environment variables are generally not propagated from client to server when
-YARN applications are launched. That is: having the AWS environment variables
-set when an application is launched will not permit the launched application
-to access S3 resources. The environment variables must (somehow) be set
-on the hosts/processes where the work is executed.
-
-### <a name="auth_providers"></a> Changing Authentication Providers
-
-The standard way to authenticate is with an access key and secret key set in
-the Hadoop configuration files.
-
-By default, the S3A client follows the following authentication chain:
-
-1. The options `fs.s3a.access.key`, `fs.s3a.secret.key` and `fs.s3a.sesson.key`
-are looked for in the Hadoop XML configuration/Hadoop credential providers,
-returning a set of session credentials if all three are defined.
-1. The `fs.s3a.access.key` and `fs.s3a.secret.key` are looked for in the Hadoop
-XML configuration//Hadoop credential providers, returning a set of long-lived
-credentials if they are defined.
-1. The [AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment),
-are then looked for: these will return session or full credentials depending
-on which values are set.
-1. An attempt is made to query the Amazon EC2 Instance/k8s container Metadata Service to
- retrieve credentials published to EC2 VMs.
-
-S3A can be configured to obtain client authentication providers from classes
-which integrate with the AWS SDK by implementing the
-`software.amazon.awssdk.auth.credentials.AwsCredentialsProvider`
-interface.
-This is done by listing the implementation classes, in order of
-preference, in the configuration option `fs.s3a.aws.credentials.provider`.
-In previous hadoop releases, providers were required to
-implement the AWS V1 SDK interface `com.amazonaws.auth.AWSCredentialsProvider`.
-Consult the [Upgrading S3A to AWS SDK V2](./aws_sdk_upgrade.html) documentation
-to see how to migrate credential providers.
-
-*Important*: AWS Credential Providers are distinct from _Hadoop Credential Providers_.
-As will be covered later, Hadoop Credential Providers allow passwords and other secrets
-to be stored and transferred more securely than in XML configuration files.
-AWS Credential Providers are classes which can be used by the Amazon AWS SDK to
-obtain an AWS login from a different source in the system, including environment
-variables, JVM properties and configuration files.
-
-All Hadoop `fs.s3a.` options used to store login details can all be secured
-in [Hadoop credential providers](../../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html);
-this is advised as a more secure way to store valuable secrets.
-
-There are a number of AWS Credential Providers inside the `hadoop-aws` JAR:
-
-| Hadoop module credential provider                              | Authentication Mechanism                         |
-|----------------------------------------------------------------|--------------------------------------------------|
-| `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider`     | Session Credentials in configuration             |
-| `org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider`        | Simple name/secret credentials in configuration  |
-| `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider`     | Anonymous Login                                  |
-| `org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider`  | [Assumed Role credentials](./assumed_roles.html) |
-| `org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider` | EC2/k8s instance credentials                     |
-
-
-There are also many in the Amazon SDKs, with the common ones being as follows
-
-| classname                                                                        | description                  |
-|----------------------------------------------------------------------------------|------------------------------|
-| `software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider` | AWS Environment Variables    |
-| `software.amazon.awssdk.auth.credentials.InstanceProfileCredentialsProvider`     | EC2 Metadata Credentials     |
-| `software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider`           | EC2/k8s Metadata Credentials |
-
-
-
-### <a name="auth_iam"></a> EC2 IAM Metadata Authentication with `InstanceProfileCredentialsProvider`
-
-Applications running in EC2 may associate an IAM role with the VM and query the
-[EC2 Instance Metadata Service](http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-instance-metadata.html)
-for credentials to access S3.  Within the AWS SDK, this functionality is
-provided by `InstanceProfileCredentialsProvider`, which internally enforces a
-singleton instance in order to prevent throttling problem.
-
-### <a name="auth_named_profile"></a> Using Named Profile Credentials with `ProfileCredentialsProvider`
-
-You can configure Hadoop to authenticate to AWS using a [named profile](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-profiles.html).
-
-To authenticate with a named profile:
-
-1. Declare `software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider` as the provider.
-1. Set your profile via the `AWS_PROFILE` environment variable.
-1. Due to a [bug in version 1 of the AWS Java SDK](https://github.com/aws/aws-sdk-java/issues/803),
-you'll need to remove the `profile` prefix from the AWS configuration section heading.
-
-    Here's an example of what your AWS configuration files should look like:
-
-    ```
-    $ cat ~/.aws/config
-    [user1]
-    region = us-east-1
-    $ cat ~/.aws/credentials
-    [user1]
-    aws_access_key_id = ...
-    aws_secret_access_key = ...
-    aws_session_token = ...
-    aws_security_token = ...
-    ```
-Note:
-
-1. The `region` setting is only used if `fs.s3a.endpoint.region` is set to the empty string.
-1. For the credentials to be available to applications running in a Hadoop cluster, the
-   configuration files MUST be in the `~/.aws/` directory on the local filesystem in
-   all hosts in the cluster.
-
-### <a name="auth_session"></a> Using Session Credentials with `TemporaryAWSCredentialsProvider`
-
-[Temporary Security Credentials](http://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp.html)
-can be obtained from the Amazon Security Token Service; these
-consist of an access key, a secret key, and a session token.
-
-To authenticate with these:
-
-1. Declare `org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider` as the
-provider.
-1. Set the session key in the property `fs.s3a.session.token`,
-and the access and secret key properties to those of this temporary session.
-
-Example:
-
-```xml
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <value>org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider</value>
-</property>
-
-<property>
-  <name>fs.s3a.access.key</name>
-  <value>SESSION-ACCESS-KEY</value>
-</property>
-
-<property>
-  <name>fs.s3a.secret.key</name>
-  <value>SESSION-SECRET-KEY</value>
-</property>
-
-<property>
-  <name>fs.s3a.session.token</name>
-  <value>SECRET-SESSION-TOKEN</value>
-</property>
-```
-
-The lifetime of session credentials are fixed when the credentials
-are issued; once they expire the application will no longer be able to
-authenticate to AWS.
-
-### <a name="auth_anon"></a> Anonymous Login with `AnonymousAWSCredentialsProvider`
-
-Specifying `org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider` allows
-anonymous access to a publicly accessible S3 bucket without any credentials.
-It can be useful for accessing public data sets without requiring AWS credentials.
-
-```xml
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <value>org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider</value>
-</property>
-```
-
-Once this is done, there's no need to supply any credentials
-in the Hadoop configuration or via environment variables.
-
-This option can be used to verify that an object store does
-not permit unauthenticated access: that is, if an attempt to list
-a bucket is made using the anonymous credentials, it should fail —unless
-explicitly opened up for broader access.
-
-```bash
-hadoop fs -ls \
- -D fs.s3a.aws.credentials.provider=org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider \
- s3a://noaa-isd-pds/
-```
-
-1. Allowing anonymous access to an S3 bucket compromises
-security and therefore is unsuitable for most use cases.
-
-1. If a list of credential providers is given in `fs.s3a.aws.credentials.provider`,
-then the Anonymous Credential provider *must* come last. If not, credential
-providers listed after it will be ignored.
-
-### <a name="auth_simple"></a> Simple name/secret credentials with `SimpleAWSCredentialsProvider`*
-
-This is the standard credential provider, which supports the secret
-key in `fs.s3a.access.key` and token in `fs.s3a.secret.key`
-values.
-
-```xml
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <value>org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider</value>
-</property>
-```
-
-This is the basic authenticator used in the default authentication chain.
-
-This means that the default S3A authentication chain can be defined as
-
-```xml
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <value>
-    org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider,
-    org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider,
-    software.amazon.awssdk.auth.credentials.EnvironmentVariableCredentialsProvider
-    org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider
-  </value>
-</property>
-```
-
-## <a name="auth_security"></a> Protecting the AWS Credentials
-
-It is critical that you never share or leak your AWS credentials.
-Loss of credentials can leak/lose all your data, run up large bills,
-and significantly damage your organisation.
-
-1. Never share your secrets.
-
-1. Never commit your secrets into an SCM repository.
-The [git secrets](https://github.com/awslabs/git-secrets) can help here.
-
-1. Never include AWS credentials in bug reports, files attached to them,
-or similar.
-
-1. If you use the `AWS_` environment variables,  your list of environment variables
-is equally sensitive.
-
-1. Never use root credentials.
-Use IAM user accounts, with each user/application having its own set of credentials.
-
-1. Use IAM permissions to restrict the permissions individual users and applications
-have. This is best done through roles, rather than configuring individual users.
-
-1. Avoid passing in secrets to Hadoop applications/commands on the command line.
-The command line of any launched program is visible to all users on a Unix system
-(via `ps`), and preserved in command histories.
-
-1. Explore using [IAM Assumed Roles](assumed_roles.html) for role-based permissions
-management: a specific S3A connection can be made with a different assumed role
-and permissions from the primary user account.
-
-1. Consider a workflow in which users and applications are issued with short-lived
-session credentials, configuring S3A to use these through
-the `TemporaryAWSCredentialsProvider`.
-
-1. Have a secure process in place for cancelling and re-issuing credentials for
-users and applications. Test it regularly by using it to refresh credentials.
-
-1. In installations where Kerberos is enabled, [S3A Delegation Tokens](delegation_tokens.html)
-can be used to acquire short-lived session/role credentials and then pass them
-into the shared application. This can ensure that the long-lived secrets stay
-on the local system.
-
-When running in EC2, the IAM EC2 instance credential provider will automatically
-obtain the credentials needed to access AWS services in the role the EC2 VM
-was deployed as.
-This AWS credential provider is enabled in S3A by default.
-
-## Custom AWS Credential Providers and Apache Spark
-
-Apache Spark employs two class loaders, one that loads "distribution" (Spark + Hadoop) classes and one that
-loads custom user classes. If the user wants to load custom implementations of AWS credential providers,
-custom signers, delegation token providers or any other dynamically loaded extension class
-through user provided jars she will need to set the following configuration:
-
-```xml
-<property>
-  <name>fs.s3a.classloader.isolation</name>
-  <value>false</value>
-</property>
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <value>CustomCredentialsProvider</value>
-</property>
-```
-
-If the following property is not set or set to `true`, the following exception will be thrown:
-
-```
-java.io.IOException: From option fs.s3a.aws.credentials.provider java.lang.ClassNotFoundException: Class CustomCredentialsProvider not found
-```
-
+See [Authenticating with S3](authentication.md).
 
 ## <a name="hadoop_credential_providers"></a>Storing secrets with Hadoop Credential Providers
 

+ 126 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AccessGrantConfiguration.java

@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.assertj.core.api.AbstractStringAssert;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.awscore.AwsClient;
+import software.amazon.awssdk.s3accessgrants.plugin.S3AccessGrantsIdentityProvider;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.AbstractHadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_ACCESS_GRANTS_ENABLED;
+
+
+/**
+ * Test S3 Access Grants configurations.
+ */
+public class TestS3AccessGrantConfiguration extends AbstractHadoopTestBase {
+  /**
+   * This credential provider will be attached to any client
+   * that has been configured with the S3 Access Grants plugin.
+   * {@code software.amazon.awssdk.s3accessgrants.plugin.S3AccessGrantsPlugin}.
+   */
+  public static final String S3_ACCESS_GRANTS_EXPECTED_CREDENTIAL_PROVIDER_CLASS =
+      S3AccessGrantsIdentityProvider.class.getName();
+
+  @Test
+  public void testS3AccessGrantsEnabled() throws IOException, URISyntaxException {
+    assertCredentialProviderClass(
+        createConfig(true),
+        true,
+        "S3 Access Grants is explicitly enabled on an S3 Async Client",
+        true);
+
+    assertCredentialProviderClass(
+        createConfig(true),
+        false,
+        "S3 Access Grants is explicitly enabled on an S3 Non-Async Client",
+        true);
+  }
+
+  @Test
+  public void testS3AccessGrantsDisabled() throws IOException, URISyntaxException {
+    assertCredentialProviderClass(
+        new Configuration(),
+        true,
+        "S3 Access Grants is implicitly disabled (default behavior) on an S3 Async Client",
+        false);
+
+    assertCredentialProviderClass(
+        new Configuration(),
+        false,
+        "S3 Access Grants is implicitly disabled (default behavior) on an S3 Non-Async Client",
+        false);
+
+    assertCredentialProviderClass(
+        createConfig(false),
+        true,
+        "S3 Access Grants is explicitly disabled on an S3 Async Client",
+        false);
+
+    assertCredentialProviderClass(
+        createConfig(false),
+        false,
+        "S3 Access Grants is explicitly disabled on an S3 Non-Async Client",
+        false);
+  }
+
+  private Configuration createConfig(boolean s3agEnabled) {
+    Configuration conf = new Configuration();
+    conf.setBoolean(AWS_S3_ACCESS_GRANTS_ENABLED, s3agEnabled);
+    return conf;
+  }
+
+  private String getCredentialProviderName(AwsClient awsClient) {
+    return awsClient.serviceClientConfiguration().credentialsProvider().getClass().getName();
+  }
+
+  private AwsClient getAwsClient(Configuration conf, boolean asyncClient)
+      throws IOException, URISyntaxException {
+    DefaultS3ClientFactory factory = new DefaultS3ClientFactory();
+    factory.setConf(conf);
+    S3ClientFactory.S3ClientCreationParameters parameters =
+        new S3ClientFactory.S3ClientCreationParameters();
+    URI uri = new URI("any-uri");
+    return asyncClient ?
+        factory.createS3AsyncClient(uri, parameters): factory.createS3Client(uri, parameters);
+  }
+
+  private void assertCredentialProviderClass(
+      Configuration configuration, boolean asyncClient, String message, boolean shouldMatch)
+      throws IOException, URISyntaxException {
+    AwsClient awsClient = getAwsClient(configuration, asyncClient);
+    AbstractStringAssert<?> assertion =
+        Assertions
+            .assertThat(S3_ACCESS_GRANTS_EXPECTED_CREDENTIAL_PROVIDER_CLASS)
+            .describedAs(message);
+    if (shouldMatch) {
+      assertion.isEqualTo(getCredentialProviderName(awsClient));
+    } else {
+      assertion.isNotEqualTo(getCredentialProviderName(awsClient));
+    }
+  }
+}