You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ex...@apache.org on 2021/10/08 17:34:42 UTC

[nifi] branch main updated: NIFI-9221: Add AWS SecretsManager Sensitive Props Provider

This is an automated email from the ASF dual-hosted git repository.

exceptionfactory pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new f3136f0  NIFI-9221: Add AWS SecretsManager Sensitive Props Provider
f3136f0 is described below

commit f3136f07ebd3609cda715056d72f19fa7bc9e706
Author: Joe Gresock <jg...@gmail.com>
AuthorDate: Wed Sep 22 13:18:43 2021 -0400

    NIFI-9221: Add AWS SecretsManager Sensitive Props Provider
    
    This closes #5410
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../nifi/properties/BootstrapProperties.java       |   2 +-
 .../nifi/properties/ProtectedPropertyContext.java  |  16 ++
 .../nifi-sensitive-property-provider/pom.xml       |  15 ++
 ...actHashiCorpVaultSensitivePropertyProvider.java |  12 +-
 ...AwsSecretsManagerSensitivePropertyProvider.java | 163 +++++++++++++++++++++
 .../nifi/properties/PropertyProtectionScheme.java  |   1 +
 .../StandardSensitivePropertyProviderFactory.java  |   9 ++
 ...rovider.java => AbstractAwsClientProvider.java} |  45 +++---
 .../configuration/AwsKmsClientProvider.java        |  64 +++-----
 .../AwsSecretsManagerClientProvider.java           |  50 +++++++
 .../configuration/AzureClientProvider.java         |  29 ----
 .../BootstrapPropertiesClientProvider.java         |  16 +-
 .../GoogleKeyManagementServiceClientProvider.java  |   7 +
 .../AwsKmsSensitivePropertyProviderIT.java         |   2 +-
 ...SecretsManagerSensitivePropertyProviderIT.java} |  43 +++---
 ...ecretsManagerSensitivePropertyProviderTest.java | 116 +++++++++++++++
 .../src/main/asciidoc/administration-guide.adoc    |  16 ++
 nifi-docs/src/main/asciidoc/toolkit-guide.adoc     |  13 +-
 .../src/main/resources/conf/bootstrap.conf         |   4 +-
 .../src/main/resources/conf/bootstrap.conf         |   4 +-
 20 files changed, 486 insertions(+), 141 deletions(-)

diff --git a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
index cc31acf..1bdf023 100644
--- a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
+++ b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
@@ -34,7 +34,7 @@ public class BootstrapProperties extends StandardReadableProperties {
     public enum BootstrapPropertyKey {
         SENSITIVE_KEY("bootstrap.sensitive.key"),
         HASHICORP_VAULT_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.hashicorp.vault.conf"),
-        AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.aws.kms.conf"),
+        AWS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.aws.conf"),
         AZURE_KEYVAULT_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.azure.keyvault.conf"),
         GCP_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.gcp.kms.conf"),
         CONTEXT_MAPPING_PREFIX("bootstrap.protection.context.mapping.");
diff --git a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedPropertyContext.java b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedPropertyContext.java
index ca2da37..0cdf46b 100644
--- a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedPropertyContext.java
+++ b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedPropertyContext.java
@@ -64,4 +64,20 @@ public class ProtectedPropertyContext {
     public String getContextKey() {
         return String.format("%s/%s", contextName, propertyName);
     }
+
+    /**
+     * Returns the property name
+     * @return The property name
+     */
+    public String getPropertyName() {
+        return propertyName;
+    }
+
+    /**
+     * Returns the context name
+     * @return The context name
+     */
+    public String getContextName() {
+        return contextName;
+    }
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/pom.xml b/nifi-commons/nifi-sensitive-property-provider/pom.xml
index 0fb83e35..aa23a5a 100644
--- a/nifi-commons/nifi-sensitive-property-provider/pom.xml
+++ b/nifi-commons/nifi-sensitive-property-provider/pom.xml
@@ -72,6 +72,21 @@
             </exclusions>
         </dependency>
         <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>secretsmanager</artifactId>
+            <version>${aws.sdk.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>software.amazon.awssdk</groupId>
+                    <artifactId>netty-nio-client</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>software.amazon.awssdk</groupId>
+                    <artifactId>apache-client</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
             <groupId>com.azure</groupId>
             <artifactId>azure-security-keyvault-secrets</artifactId>
             <version>4.3.3</version>
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AbstractHashiCorpVaultSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AbstractHashiCorpVaultSensitivePropertyProvider.java
index 63c8c62..f992ccd 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AbstractHashiCorpVaultSensitivePropertyProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AbstractHashiCorpVaultSensitivePropertyProvider.java
@@ -43,7 +43,7 @@ public abstract class AbstractHashiCorpVaultSensitivePropertyProvider extends Ab
         if (hasRequiredVaultProperties()) {
             try {
                 vaultCommunicationService = new StandardHashiCorpVaultCommunicationService(getVaultPropertySource(vaultBootstrapConfFilename));
-            } catch (IOException e) {
+            } catch (final IOException e) {
                 throw new SensitivePropertyProtectionException("Error configuring HashiCorpVaultCommunicationService", e);
             }
         } else {
@@ -64,7 +64,7 @@ public abstract class AbstractHashiCorpVaultSensitivePropertyProvider extends Ab
             try {
                 vaultBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
                         Paths.get(vaultBootstrapConfFilename), VAULT_PREFIX);
-            } catch (IOException e) {
+            } catch (final IOException e) {
                 throw new SensitivePropertyProtectionException("Could not load " + vaultBootstrapConfFilename, e);
             }
         } else {
@@ -97,14 +97,6 @@ public abstract class AbstractHashiCorpVaultSensitivePropertyProvider extends Ab
         return hasRequiredVaultProperties();
     }
 
-    /**
-     * Returns the Vault-specific bootstrap properties (e.g., bootstrap-vault.properties)
-     * @return The Vault-specific bootstrap properties
-     */
-    protected BootstrapProperties getVaultBootstrapProperties() {
-        return vaultBootstrapProperties;
-    }
-
     private boolean hasRequiredVaultProperties() {
         return vaultBootstrapProperties != null
                 && (vaultBootstrapProperties.getProperty(VaultConfigurationKey.URI.getKey()) != null)
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java
new file mode 100644
index 0000000..cb899fd
--- /dev/null
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java
@@ -0,0 +1,163 @@
+/*
+ * 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.nifi.properties;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.secretsmanager.model.SecretsManagerException;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class AwsSecretsManagerSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private final SecretsManagerClient client;
+    private final ObjectMapper objectMapper;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    AwsSecretsManagerSensitivePropertyProvider(final SecretsManagerClient client) {
+        super(null);
+
+        this.client = client;
+        this.objectMapper = new ObjectMapper();
+    }
+
+    @Override
+    public boolean isSupported() {
+        return client != null;
+    }
+
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+        Objects.requireNonNull(unprotectedValue, "Property value must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+
+        try {
+            writeLock.lock();
+            final String secretName = context.getContextName();
+            final Optional<ObjectNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            final ObjectNode secretObject = secretKeyValuesOptional.orElse(objectMapper.createObjectNode());
+
+            secretObject.put(context.getPropertyName(), unprotectedValue);
+            final String secretString = objectMapper.writeValueAsString(secretObject);
+
+            if (secretKeyValuesOptional.isPresent()) {
+                client.putSecretValue(builder -> builder.secretId(secretName).secretString(secretString));
+            } else {
+                client.createSecret(builder -> builder.name(secretName).secretString(secretString));
+            }
+            return context.getContextKey();
+        } catch (final SecretsManagerException | JsonProcessingException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Stored for [%s]", context), e);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+        try {
+            readLock.lock();
+
+            String propertyValue = null;
+            final Optional<ObjectNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            if (secretKeyValuesOptional.isPresent()) {
+                final ObjectNode secretKeyValues = secretKeyValuesOptional.get();
+                final String propertyName = context.getPropertyName();
+                if (secretKeyValues.has(propertyName)) {
+                    propertyValue = secretKeyValues.get(propertyName).textValue();
+                }
+            }
+            if (propertyValue == null) {
+                throw new SensitivePropertyProtectionException(
+                        String.format("AWS Secret Name [%s] Property Name [%s] not found", context.getContextName(), context.getPropertyName()));
+            }
+
+            return propertyValue;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    /**
+     * Returns the optional parsed JSON from the matching secret, or empty if the secret does not exist.
+     * @param context The property context
+     * @return The optional parsed JSON, or empty if the secret does not exist
+     */
+    private Optional<ObjectNode> getSecretKeyValues(final ProtectedPropertyContext context) {
+        try {
+            final GetSecretValueResponse response = client.getSecretValue(builder -> builder.secretId(context.getContextName()));
+
+            if (response.secretString() == null) {
+                throw new SensitivePropertyProtectionException(String.format("AWS Secret Name [%s] string value not found",
+                        context.getContextKey()));
+            }
+            final JsonNode responseNode = objectMapper.readTree(response.secretString());
+            if (!(responseNode instanceof ObjectNode)) {
+                throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret [%s] JSON parsing failed",
+                        context.getContextKey()));
+            }
+            return Optional.of((ObjectNode) responseNode) ;
+        } catch (final ResourceNotFoundException e) {
+            return Optional.empty();
+        }  catch (final SecretsManagerException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret [%s] retrieval failed",
+                    context.getContextKey()), e);
+        } catch (final JsonProcessingException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret [%s] JSON parsing failed",
+                    context.getContextKey()), e);
+        }
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_SECRETSMANAGER;
+    }
+
+    @Override
+    public String getIdentifierKey() {
+        return getProtectionScheme().getIdentifier();
+    }
+
+    @Override
+    public void cleanUp() {
+        if (client != null) {
+            client.close();
+        }
+    }
+}
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
index 7d71004..4a85799 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/PropertyProtectionScheme.java
@@ -25,6 +25,7 @@ import java.util.Objects;
  */
 public enum PropertyProtectionScheme {
     AES_GCM("aes/gcm/(128|192|256)", "aes/gcm/%s", "AES Sensitive Property Provider", true),
+    AWS_SECRETSMANAGER("aws/secretsmanager", "aws/secretsmanager", "AWS Secrets Manager Sensitive Property Provider", false),
     AWS_KMS("aws/kms", "aws/kms", "AWS KMS Sensitive Property Provider", false),
     AZURE_KEYVAULT_KEY("azure/keyvault/key", "azure/keyvault/key", "Azure Key Vault Key Sensitive Property Provider", false),
     AZURE_KEYVAULT_SECRET("azure/keyvault/secret", "azure/keyvault/secret", "Azure Key Vault Secret Sensitive Property Provider", false),
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
index 1d7e12f..f9a9e6c 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
@@ -21,6 +21,7 @@ import com.azure.security.keyvault.secrets.SecretClient;
 import com.google.cloud.kms.v1.KeyManagementServiceClient;
 import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
 import org.apache.nifi.properties.configuration.AwsKmsClientProvider;
+import org.apache.nifi.properties.configuration.AwsSecretsManagerClientProvider;
 import org.apache.nifi.properties.configuration.AzureCryptographyClientProvider;
 import org.apache.nifi.properties.configuration.AzureSecretClientProvider;
 import org.apache.nifi.properties.configuration.ClientProvider;
@@ -30,6 +31,7 @@ import org.apache.nifi.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -137,6 +139,13 @@ public class StandardSensitivePropertyProviderFactory implements SensitiveProper
                     final Optional<KmsClient> kmsClient = clientProvider.getClient(clientProperties);
                     return new AwsKmsSensitivePropertyProvider(kmsClient.orElse(null), clientProperties);
                 });
+            case AWS_SECRETSMANAGER:
+                return providerMap.computeIfAbsent(protectionScheme, s -> {
+                    final AwsSecretsManagerClientProvider clientProvider = new AwsSecretsManagerClientProvider();
+                    final Properties clientProperties = getClientProperties(clientProvider);
+                    final Optional<SecretsManagerClient> secretsManagerClient = clientProvider.getClient(clientProperties);
+                    return new AwsSecretsManagerSensitivePropertyProvider(secretsManagerClient.orElse(null));
+                });
             case AZURE_KEYVAULT_KEY:
                 return providerMap.computeIfAbsent(protectionScheme, s -> {
                     final AzureCryptographyClientProvider clientProvider = new AzureCryptographyClientProvider();
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsKmsClientProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AbstractAwsClientProvider.java
similarity index 66%
copy from nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsKmsClientProvider.java
copy to nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AbstractAwsClientProvider.java
index 5e01fed..9b002b6 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsKmsClientProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AbstractAwsClientProvider.java
@@ -19,28 +19,26 @@ package org.apache.nifi.properties.configuration;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.properties.BootstrapProperties;
 import org.apache.nifi.properties.SensitivePropertyProtectionException;
-
 import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
-import software.amazon.awssdk.regions.Region;
-import software.amazon.awssdk.services.kms.KmsClient;
-import software.amazon.awssdk.services.kms.KmsClientBuilder;
+import software.amazon.awssdk.core.SdkClient;
 
 import java.util.Properties;
 
 /**
- * Amazon Web Services Key Management Service Client Provider
+ * Amazon Web Services Service Client Provider base class
  */
-public class AwsKmsClientProvider extends BootstrapPropertiesClientProvider<KmsClient> {
+public abstract class AbstractAwsClientProvider<T extends SdkClient> extends BootstrapPropertiesClientProvider<T> {
     private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
 
     private static final String SECRET_KEY_PROPS_NAME = "aws.secret.access.key";
 
     private static final String REGION_KEY_PROPS_NAME = "aws.region";
 
-    public AwsKmsClientProvider() {
-        super(BootstrapProperties.BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF);
+    public AbstractAwsClientProvider() {
+        super(BootstrapProperties.BootstrapPropertyKey.AWS_SENSITIVE_PROPERTY_PROVIDER_CONF);
     }
 
     /**
@@ -50,31 +48,42 @@ public class AwsKmsClientProvider extends BootstrapPropertiesClientProvider<KmsC
      * @return KMS Client
      */
     @Override
-    protected KmsClient getConfiguredClient(final Properties clientProperties) {
+    protected T getConfiguredClient(final Properties clientProperties) {
         final String accessKey = clientProperties.getProperty(ACCESS_KEY_PROPS_NAME);
         final String secretKey = clientProperties.getProperty(SECRET_KEY_PROPS_NAME);
         final String region = clientProperties.getProperty(REGION_KEY_PROPS_NAME);
 
-        final KmsClientBuilder kmsClientBuilder = KmsClient.builder();
         if (StringUtils.isNoneBlank(accessKey, secretKey, region)) {
             logger.debug("AWS Credentials Location: Client Properties");
             try {
                 final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKey, secretKey);
-                return kmsClientBuilder
-                        .region(Region.of(region))
-                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
-                        .build();
+                return createClient(credentials, region);
             } catch (final RuntimeException e) {
-                throw new SensitivePropertyProtectionException("AWS KMS Client Builder Failed using Client Properties", e);
+                throw new SensitivePropertyProtectionException("AWS Client Builder Failed using Client Properties", e);
             }
         } else {
             logger.debug("AWS Credentials Location: Default Credentials Provider");
             try {
                 final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder().build();
-                return kmsClientBuilder.credentialsProvider(credentialsProvider).build();
+                return createDefaultClient(credentialsProvider);
             } catch (final RuntimeException e) {
-                throw new SensitivePropertyProtectionException("AWS KMS Client Builder Failed using Default Credentials Provider", e);
+                throw new SensitivePropertyProtectionException("AWS Client Builder Failed using Default Credentials Provider", e);
             }
         }
     }
+
+    /**
+     * Create a client with the given credentials and region.
+     * @param credentials AWS credentials
+     * @param region AWS region
+     * @return The created client
+     */
+    protected abstract T createClient(AwsCredentials credentials, String region);
+
+    /**
+     * Create a default client with the given credentials provider.
+     * @param credentialsProvider AWS credentials provider
+     * @return The created client
+     */
+    protected abstract T createDefaultClient(AwsCredentialsProvider credentialsProvider);
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsKmsClientProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsKmsClientProvider.java
index 5e01fed..d501735 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsKmsClientProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsKmsClientProvider.java
@@ -16,65 +16,37 @@
  */
 package org.apache.nifi.properties.configuration;
 
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.properties.BootstrapProperties;
-import org.apache.nifi.properties.SensitivePropertyProtectionException;
-
-import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
-import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
 import software.amazon.awssdk.regions.Region;
 import software.amazon.awssdk.services.kms.KmsClient;
-import software.amazon.awssdk.services.kms.KmsClientBuilder;
 
-import java.util.Properties;
+import java.util.Collections;
+import java.util.Set;
 
 /**
  * Amazon Web Services Key Management Service Client Provider
  */
-public class AwsKmsClientProvider extends BootstrapPropertiesClientProvider<KmsClient> {
-    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
-
-    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.access.key";
+public class AwsKmsClientProvider extends AbstractAwsClientProvider<KmsClient> {
 
-    private static final String REGION_KEY_PROPS_NAME = "aws.region";
+    protected static final String KEY_ID_PROPERTY = "aws.kms.key.id";
 
-    public AwsKmsClientProvider() {
-        super(BootstrapProperties.BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF);
+    @Override
+    protected KmsClient createClient(final AwsCredentials credentials, final String region) {
+        return KmsClient.builder()
+                .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                .region(Region.of(region))
+                .build();
     }
 
-    /**
-     * Get Configured Client using either Client Properties or AWS Default Credentials Provider
-     *
-     * @param clientProperties Client Properties
-     * @return KMS Client
-     */
     @Override
-    protected KmsClient getConfiguredClient(final Properties clientProperties) {
-        final String accessKey = clientProperties.getProperty(ACCESS_KEY_PROPS_NAME);
-        final String secretKey = clientProperties.getProperty(SECRET_KEY_PROPS_NAME);
-        final String region = clientProperties.getProperty(REGION_KEY_PROPS_NAME);
+    protected KmsClient createDefaultClient(final AwsCredentialsProvider credentialsProvider) {
+        return KmsClient.builder().credentialsProvider(credentialsProvider).build();
+    }
 
-        final KmsClientBuilder kmsClientBuilder = KmsClient.builder();
-        if (StringUtils.isNoneBlank(accessKey, secretKey, region)) {
-            logger.debug("AWS Credentials Location: Client Properties");
-            try {
-                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKey, secretKey);
-                return kmsClientBuilder
-                        .region(Region.of(region))
-                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
-                        .build();
-            } catch (final RuntimeException e) {
-                throw new SensitivePropertyProtectionException("AWS KMS Client Builder Failed using Client Properties", e);
-            }
-        } else {
-            logger.debug("AWS Credentials Location: Default Credentials Provider");
-            try {
-                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder().build();
-                return kmsClientBuilder.credentialsProvider(credentialsProvider).build();
-            } catch (final RuntimeException e) {
-                throw new SensitivePropertyProtectionException("AWS KMS Client Builder Failed using Default Credentials Provider", e);
-            }
-        }
+    @Override
+    protected Set<String> getRequiredPropertyNames() {
+        return Collections.singleton(KEY_ID_PROPERTY);
     }
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsSecretsManagerClientProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsSecretsManagerClientProvider.java
new file mode 100644
index 0000000..d0863dd
--- /dev/null
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AwsSecretsManagerClientProvider.java
@@ -0,0 +1,50 @@
+/*
+ * 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.nifi.properties.configuration;
+
+import software.amazon.awssdk.auth.credentials.AwsCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+
+import java.util.Collections;
+import java.util.Set;
+
+/**
+ * Amazon Web Services Secrets Manager Client Provider
+ */
+public class AwsSecretsManagerClientProvider extends AbstractAwsClientProvider<SecretsManagerClient> {
+
+    @Override
+    protected SecretsManagerClient createClient(final AwsCredentials credentials, final String region) {
+        return SecretsManagerClient.builder()
+                .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                .region(Region.of(region))
+                .build();
+    }
+
+    @Override
+    protected SecretsManagerClient createDefaultClient(final AwsCredentialsProvider credentialsProvider) {
+        return SecretsManagerClient.builder().credentialsProvider(credentialsProvider).build();
+    }
+
+    @Override
+    protected Set<String> getRequiredPropertyNames() {
+        return Collections.emptySet();
+    }
+}
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AzureClientProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AzureClientProvider.java
index 61561e7..bc48e45 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AzureClientProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AzureClientProvider.java
@@ -19,11 +19,6 @@ package org.apache.nifi.properties.configuration;
 import com.azure.core.credential.TokenCredential;
 import com.azure.identity.DefaultAzureCredentialBuilder;
 import org.apache.nifi.properties.BootstrapProperties;
-import org.apache.nifi.util.StringUtils;
-
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
 
 /**
  * Abstract Microsoft Azure Client Provider
@@ -34,17 +29,6 @@ public abstract class AzureClientProvider<T> extends BootstrapPropertiesClientPr
     }
 
     /**
-     * Get Client using Client Properties
-     *
-     * @param clientProperties Client Properties can be null
-     * @return Configured Client or empty when Client Properties object is null
-     */
-    @Override
-    public Optional<T> getClient(final Properties clientProperties) {
-        return isMissingProperties(clientProperties) ? Optional.empty() : Optional.of(getConfiguredClient(clientProperties));
-    }
-
-    /**
      * Get Default Azure Token Credential using Default Credentials Builder for environment variables and system properties
      *
      * @return Token Credential
@@ -52,17 +36,4 @@ public abstract class AzureClientProvider<T> extends BootstrapPropertiesClientPr
     protected TokenCredential getDefaultTokenCredential() {
         return new DefaultAzureCredentialBuilder().build();
     }
-
-    /**
-     * Get Property Names required for initializing client in order to perform initial validation
-     *
-     * @return Set of required client property names
-     */
-    protected abstract Set<String> getRequiredPropertyNames();
-
-    private boolean isMissingProperties(final Properties clientProperties) {
-        return clientProperties == null || getRequiredPropertyNames().stream().anyMatch(propertyName ->
-            StringUtils.isBlank(clientProperties.getProperty(propertyName))
-        );
-    }
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/BootstrapPropertiesClientProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/BootstrapPropertiesClientProvider.java
index 1bbb021..b3bfbd7 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/BootstrapPropertiesClientProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/BootstrapPropertiesClientProvider.java
@@ -30,6 +30,7 @@ import java.nio.file.Paths;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.Properties;
+import java.util.Set;
 
 /**
  * Shared Client Provider for reading Client Properties from file referenced in configured Bootstrap Property Key
@@ -53,7 +54,7 @@ public abstract class BootstrapPropertiesClientProvider<T> implements ClientProv
      */
     @Override
     public Optional<T> getClient(final Properties clientProperties) {
-        return clientProperties == null ? Optional.empty() : Optional.of(getConfiguredClient(clientProperties));
+        return isMissingProperties(clientProperties) ? Optional.empty() : Optional.of(getConfiguredClient(clientProperties));
     }
 
     /**
@@ -96,4 +97,17 @@ public abstract class BootstrapPropertiesClientProvider<T> implements ClientProv
      * @return Configured Client
      */
     protected abstract T getConfiguredClient(final Properties clientProperties);
+
+    /**
+     * Get Property Names required for initializing client in order to perform initial validation
+     *
+     * @return Set of required client property names
+     */
+    protected abstract Set<String> getRequiredPropertyNames();
+
+    private boolean isMissingProperties(final Properties clientProperties) {
+        return clientProperties == null || getRequiredPropertyNames().stream().anyMatch(propertyName ->
+                StringUtils.isBlank(clientProperties.getProperty(propertyName))
+        );
+    }
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/GoogleKeyManagementServiceClientProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/GoogleKeyManagementServiceClientProvider.java
index 8f46edc..2bb707d 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/GoogleKeyManagementServiceClientProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/GoogleKeyManagementServiceClientProvider.java
@@ -22,7 +22,9 @@ import org.apache.nifi.properties.BootstrapProperties;
 import org.apache.nifi.properties.SensitivePropertyProtectionException;
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Properties;
+import java.util.Set;
 
 /**
  * Google Key Management Service Client Provider
@@ -46,4 +48,9 @@ public class GoogleKeyManagementServiceClientProvider extends BootstrapPropertie
             throw new SensitivePropertyProtectionException("Google Key Management Service Create Failed", e);
         }
     }
+
+    @Override
+    protected Set<String> getRequiredPropertyNames() {
+        return Collections.emptySet();
+    }
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsKmsSensitivePropertyProviderIT.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsKmsSensitivePropertyProviderIT.java
index 708f54e..30c183c 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsKmsSensitivePropertyProviderIT.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsKmsSensitivePropertyProviderIT.java
@@ -61,7 +61,7 @@ public class AwsKmsSensitivePropertyProviderIT {
     private static final String REGION_KEY_PROPS_NAME = "aws.region";
     private static final String KMS_KEY_PROPS_NAME = "aws.kms.key.id";
 
-    private static final String BOOTSTRAP_AWS_FILE_PROPS_NAME = "nifi.bootstrap.protection.aws.kms.conf";
+    private static final String BOOTSTRAP_AWS_FILE_PROPS_NAME = "nifi.bootstrap.protection.aws.conf";
 
     private static final String EMPTY_PROPERTY = "";
 
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsKmsSensitivePropertyProviderIT.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProviderIT.java
similarity index 76%
copy from nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsKmsSensitivePropertyProviderIT.java
copy to nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProviderIT.java
index 708f54e..f60a80a 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsKmsSensitivePropertyProviderIT.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProviderIT.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.properties;
 
-import org.apache.nifi.properties.configuration.AwsKmsClientProvider;
+import org.apache.nifi.properties.configuration.AwsSecretsManagerClientProvider;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -24,7 +24,7 @@ import org.junit.Test;
 import org.mockito.internal.util.io.IOUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -36,42 +36,34 @@ import java.util.Properties;
  * To run this test, make sure to first configure sensitive credential information as in the following link
  * https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-files.html
  *
- * If you don't have a key then run:
- * aws kms create-key
- *
- * Take note of the key id or arn.
- *
- * Then, set the system property -Daws.kms.key.id to the either key id value or arn value
- *
  * The following settings are optional. If you have a default AWS configuration and credentials in ~/.aws then
  * it will take that. Otherwise you can set all of the following:
  * set the system property -Daws.access.key.id to the access key id
  * set the system property -Daws.secret.access.key to the secret access key
  * set the system property -Daws.region to the region
  *
- * After you are satisfied with the test, and you don't need the key, you may schedule key deletion with:
- * aws kms schedule-key-deletion --key-id "key id" --pending-window-in-days "number of days"
+ * After you are satisfied with the test, and you don't need the key, you may schedule secret deletion with:
+ * aws secretsmanager delete-secret --secret-id "default/property" --recovery-window-in-days 14
  *
  */
 
-public class AwsKmsSensitivePropertyProviderIT {
-    private static final String SAMPLE_PLAINTEXT = "AWSKMSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
+public class AwsSecretsManagerSensitivePropertyProviderIT {
+    private static final String SAMPLE_PLAINTEXT = "AWSSecretsManagerSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
     private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
     private static final String SECRET_KEY_PROPS_NAME = "aws.secret.access.key";
     private static final String REGION_KEY_PROPS_NAME = "aws.region";
-    private static final String KMS_KEY_PROPS_NAME = "aws.kms.key.id";
 
-    private static final String BOOTSTRAP_AWS_FILE_PROPS_NAME = "nifi.bootstrap.protection.aws.kms.conf";
+    private static final String BOOTSTRAP_AWS_FILE_PROPS_NAME = "nifi.bootstrap.protection.aws.conf";
 
     private static final String EMPTY_PROPERTY = "";
 
-    private static AwsKmsSensitivePropertyProvider spp;
+    private static AwsSecretsManagerSensitivePropertyProvider spp;
 
     private static BootstrapProperties props;
 
     private static Path mockBootstrapConf, mockAWSBootstrapConf;
 
-    private static final Logger logger = LoggerFactory.getLogger(AwsKmsSensitivePropertyProviderIT.class);
+    private static final Logger logger = LoggerFactory.getLogger(AwsSecretsManagerSensitivePropertyProviderIT.class);
 
     private static void initializeBootstrapProperties() throws IOException{
         mockBootstrapConf = Files.createTempFile("bootstrap", ".conf").toAbsolutePath();
@@ -87,14 +79,12 @@ public class AwsKmsSensitivePropertyProviderIT {
         String accessKey = System.getProperty(ACCESS_KEY_PROPS_NAME, EMPTY_PROPERTY);
         String secretKey = System.getProperty(SECRET_KEY_PROPS_NAME, EMPTY_PROPERTY);
         String region = System.getProperty(REGION_KEY_PROPS_NAME, EMPTY_PROPERTY);
-        String keyId = System.getProperty(KMS_KEY_PROPS_NAME, EMPTY_PROPERTY);
 
         StringBuilder bootstrapConfText = new StringBuilder();
         String lineSeparator = System.getProperty("line.separator");
         bootstrapConfText.append(ACCESS_KEY_PROPS_NAME + "=" + accessKey);
         bootstrapConfText.append(lineSeparator + SECRET_KEY_PROPS_NAME + "=" + secretKey);
         bootstrapConfText.append(lineSeparator + REGION_KEY_PROPS_NAME + "=" + region);
-        bootstrapConfText.append(lineSeparator + KMS_KEY_PROPS_NAME + "=" + keyId);
 
         IOUtil.writeText(bootstrapConfText.toString(), mockAWSBootstrapConf.toFile());
     }
@@ -103,10 +93,10 @@ public class AwsKmsSensitivePropertyProviderIT {
     public static void initOnce() throws IOException {
         initializeBootstrapProperties();
         Assert.assertNotNull(props);
-        final AwsKmsClientProvider provider = new AwsKmsClientProvider();
+        final AwsSecretsManagerClientProvider provider = new AwsSecretsManagerClientProvider();
         final Properties properties = provider.getClientProperties(props).orElse(null);
-        final KmsClient kmsClient = provider.getClient(properties).orElse(null);
-        spp = new AwsKmsSensitivePropertyProvider(kmsClient, properties);
+        final SecretsManagerClient secretsManagerClient = provider.getClient(properties).orElse(null);
+        spp = new AwsSecretsManagerSensitivePropertyProvider(secretsManagerClient);
         Assert.assertNotNull(spp);
     }
 
@@ -120,16 +110,17 @@ public class AwsKmsSensitivePropertyProviderIT {
 
     @Test
     public void testEncryptDecrypt() {
-        logger.info("Running testEncryptDecrypt of AWS KMS SPP integration test");
+        logger.info("Running testEncryptDecrypt of AWS Secrets Manager SPP integration test");
         runEncryptDecryptTest();
-        logger.info("testEncryptDecrypt of AWS KMS SPP integration test completed");
+        logger.info("testEncryptDecrypt of AWS Secrets Manager SPP integration test completed");
     }
 
     private static void runEncryptDecryptTest() {
+        final String propertyName = "property2";
         logger.info("Plaintext: " + SAMPLE_PLAINTEXT);
-        String protectedValue = spp.protect(SAMPLE_PLAINTEXT, ProtectedPropertyContext.defaultContext("property"));
+        String protectedValue = spp.protect(SAMPLE_PLAINTEXT, ProtectedPropertyContext.defaultContext(propertyName));
         logger.info("Protected Value: " + protectedValue);
-        String unprotectedValue = spp.unprotect(protectedValue, ProtectedPropertyContext.defaultContext("property"));
+        String unprotectedValue = spp.unprotect(protectedValue, ProtectedPropertyContext.defaultContext(propertyName));
         logger.info("Unprotected Value: " + unprotectedValue);
 
         Assert.assertEquals(SAMPLE_PLAINTEXT, unprotectedValue);
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProviderTest.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProviderTest.java
new file mode 100644
index 0000000..93addaa
--- /dev/null
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProviderTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.nifi.properties;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+import software.amazon.awssdk.services.secretsmanager.model.CreateSecretResponse;
+import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.PutSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.ResourceNotFoundException;
+
+import java.nio.charset.Charset;
+import java.util.function.Consumer;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+public class AwsSecretsManagerSensitivePropertyProviderTest {
+    private static final String PROPERTY_NAME = "propertyName";
+    private static final String PROPERTY_VALUE = "propertyValue";
+
+    private static final String SECRET_STRING = String.format("{ \"%s\": \"%s\" }", PROPERTY_NAME, PROPERTY_VALUE);
+
+    @Mock
+    private SecretsManagerClient secretsManagerClient;
+
+    private AwsSecretsManagerSensitivePropertyProvider provider;
+
+    @BeforeEach
+    public void setProvider() {
+        provider = new AwsSecretsManagerSensitivePropertyProvider(secretsManagerClient);
+    }
+
+    @Test
+    public void testValidateClientNull() {
+        final AwsSecretsManagerSensitivePropertyProvider provider = new AwsSecretsManagerSensitivePropertyProvider(null);
+        assertNotNull(provider);
+    }
+
+    @Test
+    public void testValidateKeyNoSecretString() {
+        final GetSecretValueResponse getSecretValueResponse = GetSecretValueResponse.builder()
+                .secretBinary(SdkBytes.fromString("binary", Charset.defaultCharset())).build();
+        when(secretsManagerClient.getSecretValue(any(Consumer.class))).thenReturn(getSecretValueResponse);
+
+        assertThrows(SensitivePropertyProtectionException.class, () ->
+                provider.unprotect("anyValue", ProtectedPropertyContext.defaultContext(PROPERTY_NAME)));
+    }
+
+    @Test
+    public void testCleanUp() {
+        provider.cleanUp();
+        verify(secretsManagerClient).close();
+    }
+
+    @Test
+    public void testProtectCreateSecret() {
+        final String secretName = ProtectedPropertyContext.defaultContext(PROPERTY_NAME).getContextKey();
+
+        when(secretsManagerClient.getSecretValue(any(Consumer.class))).thenThrow(ResourceNotFoundException.builder().message("Not found").build());
+
+        final CreateSecretResponse createSecretResponse = CreateSecretResponse.builder()
+                .name(secretName).build();
+        when(secretsManagerClient.createSecret(any(Consumer.class))).thenReturn(createSecretResponse);
+
+        final String protectedProperty = provider.protect(PROPERTY_VALUE, ProtectedPropertyContext.defaultContext(PROPERTY_NAME));
+        assertEquals(secretName, protectedProperty);
+    }
+
+    @Test
+    public void testProtectExistingSecret() {
+        final String secretName = ProtectedPropertyContext.defaultContext(PROPERTY_NAME).getContextKey();
+        final GetSecretValueResponse getSecretValueResponse = GetSecretValueResponse.builder().secretString(SECRET_STRING).build();
+        when(secretsManagerClient.getSecretValue(any(Consumer.class))).thenReturn(getSecretValueResponse);
+
+        final PutSecretValueResponse putSecretValueResponse = PutSecretValueResponse.builder()
+                .name(secretName).build();
+        when(secretsManagerClient.putSecretValue(any(Consumer.class))).thenReturn(putSecretValueResponse);
+
+        final String protectedProperty = provider.protect(PROPERTY_VALUE, ProtectedPropertyContext.defaultContext(PROPERTY_NAME));
+        assertEquals(secretName, protectedProperty);
+    }
+
+    @Test
+    public void testUnprotect() {
+        final GetSecretValueResponse getSecretValueResponse = GetSecretValueResponse.builder().secretString(SECRET_STRING).build();
+        when(secretsManagerClient.getSecretValue(any(Consumer.class))).thenReturn(getSecretValueResponse);
+
+        final String property = provider.unprotect("anyValue", ProtectedPropertyContext.defaultContext(PROPERTY_NAME));
+        assertEquals(PROPERTY_VALUE, property);
+    }
+}
diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc
index 18b7247..6e02e5e 100644
--- a/nifi-docs/src/main/asciidoc/administration-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc
@@ -1797,6 +1797,7 @@ For more information, see the <<toolkit-guide.adoc#encrypt_config_tool,Encrypt-C
 
 Configuring each Sensitive Property Provider requires including the appropriate file reference property in `bootstrap.conf`. The default `bootstrap.conf` includes commented file reference properties for available providers.
 
+[[hashicorp-vault-providers]]
 === HashiCorp Vault providers
 Two encryption providers are currently configurable in the `bootstrap-hashicorp-vault.conf` file:
 
@@ -1838,6 +1839,7 @@ Following are the configuration properties available inside the `bootstrap-hashi
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+[[aws-kms-provider]]
 === AWS KMS provider
 This provider uses https://aws.amazon.com/kms/[AWS Key Management Service] for decryption. AWS KMS configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in `bootstrap.conf`. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.
 
@@ -1858,6 +1860,20 @@ This provider uses https://aws.amazon.com/kms/[AWS Key Management Service] for d
 |`aws.secret.access.key`|The secret access key used to access AWS KMS.|_none_
 |===
 
+[[aws-secrets-manager-provider]]
+=== AWS Secrets Manager provider
+This provider uses https://aws.amazon.com/secrets-manager/[AWS Secrets Manager Service] to store and retrieve AWS Secrets. AWS Secrets Manager configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in `bootstrap.conf`. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.
+
+==== Optional properties
+===== All of the following must be configured, or will be ignored entirely.
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.region`|The AWS region used to configure the AWS Secrets Manager Client.|_none_
+|`aws.access.key.id`|The access key ID credential used to access AWS Secrets Manager.|_none_
+|`aws.secret.access.key`|The secret access key used to access AWS Secrets Manager.|_none_
+|===
+
 [[azure-key-vault-key-provider]]
 === Azure Key Vault Key Provider
 This protection scheme uses keys managed by
diff --git a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
index d1c74d9..6318393 100644
--- a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
@@ -435,7 +435,7 @@ The following are available options when targeting NiFi:
 * `-f`,`--flowXml <file>`                       The _flow.xml.gz_ file currently protected with old password (will be overwritten unless `-g` is specified)
 * `-g`,`--outputFlowXml <file>`                 The destination _flow.xml.gz_ file containing protected config values (will not modify input _flow.xml.gz_)
 * `-b`,`--bootstrapConf <file>`                 The bootstrap.conf file to persist root key and to optionally provide any configuration for the protection scheme.
-* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>, <<AZURE_KEYVAULT_SECRET>>, <<GCP_KMS>>] (default is AES_GCM)
+* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AWS_SECRETSMANAGER>>, <<AZURE_KEYVAULT_KEY>>, <<AZURE_KEYVAULT_SECRET>>, <<GCP_KMS>>] (default is AES_GCM)
 * `-k`,`--key <keyhex>`                         The raw hexadecimal key to use to encrypt the sensitive properties
 * `-e`,`--oldKey <keyhex>`                      The old raw hexadecimal key to use during key migration
 * `-H`,`--oldProtectionScheme <protectionScheme>` The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM
@@ -456,7 +456,7 @@ The following are available options when targeting NiFi Registry using the `--ni
 * `-v`,`--verbose`                              Sets verbose mode (default false)
 * `-p`,`--password <password>`                  Protect the files using a password-derived key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the password.
 * `-k`,`--key <keyhex>`                         Protect the files using a raw hexadecimal key. If an argument is not provided to this flag, interactive mode will be triggered to prompt the user to enter the key.
-* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AZURE_KEYVAULT_KEY>>, <<AZURE_KEYVAULT_SECRET>>, <<GCP_KMS>>]  (default is AES_GCM)
+* `-S`,`--protectionScheme <protectionScheme>`  Selects the protection scheme for encrypted properties.  Valid values are: [<<AES_GCM>>, <<HASHICORP_VAULT_TRANSIT>>, <<HASHICORP_VAULT_KV>>, <<AWS_KMS>>, <<AWS_SECRETSMANAGER>>, <<AZURE_KEYVAULT_KEY>>, <<AZURE_KEYVAULT_SECRET>>, <<GCP_KMS>>]  (default is AES_GCM)
 * `--oldPassword <password>`                    If the input files are already protected using a password-derived key, this specifies the old password so that the files can be unprotected before re-protecting.
 * `--oldKey <keyhex>`                           If the input files are already protected using a key, this specifies the raw hexadecimal key so that the files can be unprotected before re-protecting.
 * `-H`,`--oldProtectionScheme <protectionScheme>`The old protection scheme to use during encryption migration (see --protectionScheme for possible values).  Default is AES_GCM.
@@ -477,13 +477,16 @@ The protection scheme can be selected during encryption using the `--protectionS
 The default protection scheme, `AES-G/CM` simply encrypts sensitive properties and marks their protection as either `aes/gcm/256` or `aes/gcm/256` as appropriate.  This protection is all done within NiFi itself.
 
 ==== HASHICORP_VAULT_TRANSIT [[HASHICORP_VAULT_TRANSIT]]
-This protection scheme uses https://www.vaultproject.io/docs/secrets/transit[HashiCorp Vault Transit Secrets Engine] to outsource encryption to a configured Vault server. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_TRANSIT protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `boot [...]
+This protection scheme uses https://www.vaultproject.io/docs/secrets/transit[HashiCorp Vault Transit Secrets Engine] to outsource encryption to a configured Vault server. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_TRANSIT protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `boot [...]
 
 ==== HASHICORP_VAULT_KV [[HASHICORP_VAULT_KV]]
-This protection scheme uses https://www.vaultproject.io/docs/secrets/kv/kv-v1[HashiCorp Vault Key Value Secrets Engine Version 1] to store sensitive values as Vault Secrets. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_KV protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `bootst [...]
+This protection scheme uses https://www.vaultproject.io/docs/secrets/kv/kv-v1[HashiCorp Vault Key Value Secrets Engine Version 1] to store sensitive values as Vault Secrets. All HashiCorp Vault configuration is stored in the `bootstrap-hashicorp-vault.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance.  Therefore, when using the HASHICORP_VAULT_KV protection scheme, the `nifi(.registry)?.bootstrap.protection.hashicorp.vault.conf` property in the `bootst [...]
 
 ==== AWS_KMS [[AWS_KMS]]
-This protection scheme uses https://aws.amazon.com/kms/[AWS Key Management] Service for encryption and decryption. AWS KMS configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.  Therefore, wh [...]
+This protection scheme uses https://aws.amazon.com/kms/[AWS Key Management] Service for encryption and decryption. AWS KMS configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables and system properties.  Therefore, wh [...]
+
+==== AWS_SECRETSMANAGER [[AWS_SECRETSMANAGER]]
+This protection scheme uses https://aws.amazon.com/secrets-manager/[AWS Secrets Manager] Service to store sensitive values as AWS Secrets. AWS Secrets Manager configuration properties can be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of NiFi or NiFi Registry. If the configuration properties are not specified in `bootstrap-aws.conf`, then the provider will attempt to use the AWS default credentials provider, which checks standard environment variables a [...]
 
 ==== Microsoft Azure Key Vault Sensitive Property Providers
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
index 33843d7..5664cfa 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap.conf
@@ -63,8 +63,8 @@ nifi.bootstrap.sensitive.key=
 # HashiCorp Vault Sensitive Property Providers
 #nifi.bootstrap.protection.hashicorp.vault.conf=./conf/bootstrap-hashicorp-vault.conf
 
-# AWS KMS Sensitive Property Providers
-#nifi.bootstrap.protection.aws.kms.conf=./conf/bootstrap-aws.conf
+# AWS Sensitive Property Providers
+#nifi.bootstrap.protection.aws.conf=./conf/bootstrap-aws.conf
 
 # Azure Key Vault Sensitive Property Providers
 #nifi.bootstrap.protection.azure.keyvault.conf=./conf/bootstrap-azure.conf
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
index 997a05e..0fc45d5 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
+++ b/nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap.conf
@@ -58,8 +58,8 @@ nifi.registry.bootstrap.sensitive.key=
 # HashiCorp Vault Sensitive Property Providers
 #nifi.registry.bootstrap.protection.hashicorp.vault.conf=./conf/bootstrap-hashicorp-vault.conf
 
-# AWS KMS Sensitive Property Providers
-#nifi.registry.bootstrap.protection.aws.kms.conf=./conf/bootstrap-aws.conf
+# AWS Sensitive Property Providers
+#nifi.registry.bootstrap.protection.aws.conf=./conf/bootstrap-aws.conf
 
 # Azure Key Vault Sensitive Property Providers
 #nifi.registry.bootstrap.protection.azure.keyvault.conf=./conf/bootstrap-azure.conf