You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/07/06 23:32:15 UTC

[GitHub] [nifi] emiliosetiadarma opened a new pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

emiliosetiadarma opened a new pull request #5202:
URL: https://github.com/apache/nifi/pull/5202


   <!--
     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.
   -->
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   - Added AWS Sensitive Property Provider as well as Integration Test
   - Updated Toolkit Guide with PropertyProtectionScheme migration example
   - Updated SensitivePropertyProvider interface to implement the close function to close resources that a Sensitive Property Provider might have opened.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [x] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [x] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [x] Have you written or updated unit tests to verify your changes?
   - [x] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [x] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669099120



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use credentials/configuration in bootstrap-aws.conf
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain
+     * Note: This does not verify if credentials are valid
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return True if bootstrap-aws.conf contains the required properties for AWS SPP, False otherwise
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value
+     * @return the value to persist in the {@code nifi.properties} file
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }

Review comment:
       Making the changes! Moving this to another checkClientAndInitialize() function.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use credentials/configuration in bootstrap-aws.conf
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain
+     * Note: This does not verify if credentials are valid
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (IOException e) {

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668960062



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5202:
URL: https://github.com/apache/nifi/pull/5202


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669624024



##########
File path: nifi-assembly/NOTICE
##########
@@ -1013,6 +1013,13 @@ The following binary components are provided under the Apache Software License v
       Since product implements StAX API, it has dependencies to StAX API
       classes.
 
+  (ASLv2) AWS SDK for Java 2.0
+      The following NOTICE information applies:
+        Copyright 2010-2019 Amazon.com, Inc. or its affiliates. All Rights Reserved.

Review comment:
       Since this is a new entry, recommend updating the year to 2021:
   ```suggestion
           Copyright 2010-2021 Amazon.com, Inc. or its affiliates. All Rights Reserved.
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java
##########
@@ -257,4 +257,12 @@ public static int getMinCipherTextLength() {
     public static String getDelimiter() {
         return DELIMITER;
     }
+
+    /**
+     * No cleanup necessary
+     */
+    @Override
+    public void cleanUp() {
+        return;

Review comment:
       The `return` statement is unnecessary for a method with a return of `void`.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false

Review comment:
       This comment is not quite clear at this particular line, it might be better to include it on the `isSupported()` method comment block.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");

Review comment:
       Following the comment above, recommend the following adjusted wording:
   ```suggestion
               logger.debug("Using AWS credentials from default credentials provider");
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");

Review comment:
       Recommend adjusting the wording without reference to the particular filename:
   ```suggestion
               logger.debug("Using AWS credentials from bootstrap properties");
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       It doesn't seem necessary to log an error and throw an exception, since the exception will be logged higher in the call chain.  Recommend removing the log.
   ```suggestion
                   throw new SensitivePropertyProtectionException("KMS Client initialization failed using bootstrap properties", e);
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }

Review comment:
       Since this is a private method, recommend removing the null check and adding an `Objects.requireNonNull()` reference in the class constructor prior to calling this method.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");

Review comment:
       Recommend adjusting the message to avoid reference the specific filename in case the actual name is different in some cases.
   ```suggestion
               logger.warn("AWS KMS properties file path not configured in Bootstrap properties");
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");

Review comment:
       This could generate a number of log statements, it could be removed, or at least changed to a parameterized message string.
   

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a null/empty cipher");

Review comment:
       ```suggestion
               throw new IllegalArgumentException("Cannot decrypt a blank value");
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/main/java/org/apache/nifi/properties/NiFiPropertiesLoader.java
##########
@@ -173,8 +173,14 @@ public NiFiProperties load(final File file) {
                     .getSupportedSensitivePropertyProviders()
                     .forEach(protectedNiFiProperties::addSensitivePropertyProvider);
         }
-
-        return protectedNiFiProperties.getUnprotectedProperties();
+        NiFiProperties props = protectedNiFiProperties.getUnprotectedProperties();
+        if (protectedNiFiProperties.hasProtectedKeys()) {
+            // releases resources used by SPP

Review comment:
       This comment does not seem necessary since the loop calls the `cleanUp()` method.
   ```suggestion
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a null/empty cipher");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);
+            final byte[] plainBytes = decrypt(cipherBytes);
+            logger.debug(getName() + " decrypted a sensitive value successfully");

Review comment:
       Recommend removing this log as above since debugging could probably be enabled for the KMS client itself.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");

Review comment:
       Recommend adjusting the wording so that it is bit clearer and does not use the variable name:
   ```suggestion
               logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {

Review comment:
       Multiple strings can be checked using `isNoneBlank()`:
   ```suggestion
           if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {

Review comment:
       Recommend adjusting the naming to `isClientConfigured()` or `isClientInitialized()` to reflect the check more accurately. On the other hand, since it is just a null check, it may be better to just remove the method and use an inline check where needed.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {

Review comment:
       The `final` modifier is not necessary on `private` methods, so it should be removed here and from other `private` methods.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {

Review comment:
       Recommend adjusting this to catch the more generic `RuntimeException`:
   ```suggestion
               } catch (final KmsException | RuntimeException e) {
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       As above, recommend avoiding the error log and just throw the exception:
   ```suggestion
                   throw new SensitivePropertyProtectionException("KMS Client initialization failed using default credentials provider", e);
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException

Review comment:
       I'm not sure this particular comment is necessary given the exception that ends up being thrown.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);

Review comment:
       As recommend elsewhere, recommend removing the log message.
   ```suggestion
               throw new SensitivePropertyProtectionException("KMS Client not initialized");
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);

Review comment:
       ```suggestion
               throw new SensitivePropertyProtectionException("KMS Key Identifier not configured");
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();

Review comment:
       The implementation is straightforward enough that the comments do not seem necessary.
   ```suggestion
           final EncryptRequest encryptRequest = EncryptRequest.builder()
                   .keyId(keyId)
                   .plaintext(plainBytes)
                   .build();
   
           final EncryptResponse response = client.encrypt(encryptRequest);
           final SdkBytes encryptedData = response.ciphertextBlob();
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();

Review comment:
       ```suggestion
           final DecryptRequest decryptRequest = DecryptRequest.builder()
                   .ciphertextBlob(cipherBytes)
                   .keyId(keyId)
                   .build();
                   
           final DecryptResponse response = client.decrypt(decryptRequest);
           final SdkBytes decryptedData = response.plaintext();
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {

Review comment:
       As mentioned else where, it seems clearer to change this to a null check:
   ```suggestion
           if (client == null) {
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");

Review comment:
       ```suggestion
               throw new IllegalArgumentException("Cannot encrypt a blank value");
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }

Review comment:
       Since the calling method checks whether `props` is null, this separate method seems unnecessary.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);

Review comment:
       ```suggestion
                   throw new SensitivePropertyProtectionException("KMS Client initialization failed", e);
   ```

##########
File path: nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
##########
@@ -75,6 +75,13 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-mock</artifactId>
             <version>1.14.0-SNAPSHOT</version>
             <scope>test</scope>
+<!--The following is added to resolve dependencies issues bringing in AWS KMS dependency-->

Review comment:
       Recommend adjusting this comment to indicate the direct and transitive dependency more clearly:
   ```suggestion
               <!-- Exclude transitive dependency of software.amazon.awssdk:kms included in nifi-sensitive-property-provider -->
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
##########
@@ -105,6 +105,9 @@ public SensitivePropertyProvider getProvider(final PropertyProtectionScheme prot
         switch (protectionScheme) {
             case AES_GCM:
                 return providerMap.computeIfAbsent(protectionScheme, s -> new AESSensitivePropertyProvider(keyHex));
+            // Other providers may choose to pass getBootstrapProperties() into the constructor

Review comment:
       This comment could be removed.
   ```suggestion
   ```

##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service, or AWS KMS for short (https://aws.amazon.com/kms/) for encryption/decryption. All AWS KMS configuration/credentials details are to be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance. If the configuration/credentials details are not fully specified in `bootstrap-aws.conf`, then the protection scheme will attempt to use the default AWS credentials/configuration chain. Therefore, when using the AWS_KMS protection scheme, the `nifi(.registry)?.bootstrap.protection.aws.kms.conf` property in the `bootstrap.conf` specified using the `-b` flag must be available to the Encrypt Configuration Tool and must be configured as follows:

Review comment:
       Recommend adjusting the wording to clarify that providing properties in `bootstrap-aws.conf` is optional depending on the system environment configuration.
   ```suggestion
   This protection scheme uses AWS Key Management Service (https://aws.amazon.com/kms/) 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.
   ```

##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service, or AWS KMS for short (https://aws.amazon.com/kms/) for encryption/decryption. All AWS KMS configuration/credentials details are to be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance. If the configuration/credentials details are not fully specified in `bootstrap-aws.conf`, then the protection scheme will attempt to use the default AWS credentials/configuration chain. Therefore, when using the AWS_KMS protection scheme, the `nifi(.registry)?.bootstrap.protection.aws.kms.conf` property in the `bootstrap.conf` specified using the `-b` flag must be available to the Encrypt Configuration Tool and must be configured as follows:
+
+===== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.kms.key.id`|The key id or ARN to be used by AWS KMS to identify the key used for encryption/decryption.|_none_
+|===
+
+===== Optional properties
+====== All of the following must be configured, or will be ignored entirely.
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.region`|The region to configure AWS KMS Client with for encryption/decryption.|_none_

Review comment:
       ```suggestion
   |`aws.region`|The AWS region used to configure the AWS KMS Client.|_none_
   ```

##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service, or AWS KMS for short (https://aws.amazon.com/kms/) for encryption/decryption. All AWS KMS configuration/credentials details are to be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance. If the configuration/credentials details are not fully specified in `bootstrap-aws.conf`, then the protection scheme will attempt to use the default AWS credentials/configuration chain. Therefore, when using the AWS_KMS protection scheme, the `nifi(.registry)?.bootstrap.protection.aws.kms.conf` property in the `bootstrap.conf` specified using the `-b` flag must be available to the Encrypt Configuration Tool and must be configured as follows:
+
+===== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.kms.key.id`|The key id or ARN to be used by AWS KMS to identify the key used for encryption/decryption.|_none_

Review comment:
       ```suggestion
   |`aws.kms.key.id`|The identifier or ARN that the AWS KMS client uses for encryption and decryption.|_none_
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a null/empty cipher");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);
+            final byte[] plainBytes = decrypt(cipherBytes);
+            logger.debug(getName() + " decrypted a sensitive value successfully");
+            return new String(plainBytes, PROPERTY_CHARSET);
+        } catch (final SdkClientException | KmsException | DecoderException e) {
+            final String msg = "Error decrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       ```suggestion
               throw new SensitivePropertyProtectionException("Decrypt failed", e);
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java
##########
@@ -56,4 +56,10 @@
      * @return the raw value to be used by the application
      */
     String unprotect(String protectedValue) throws SensitivePropertyProtectionException;
+
+    /**
+     * Cleans up resources that may have been allocated/used by an SPP implementation
+     * Note: If there is nothing to be done, then this function is a no-op (i.e. no cleanup necessary)

Review comment:
       This particular line depends on the implementation, so it should be removed from the interface documentation.
   ```suggestion
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       ```suggestion
               throw new SensitivePropertyProtectionException("Encrypt failed", e);
   ```

##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -699,6 +719,39 @@ for each phase (old vs. new), and any combination is sufficient:
 In order to change the protection scheme (e.g., migrating from AES encryption to Vault encryption), specify the `--protectionScheme`
 and `--oldProtectionScheme` in the migration command.
 
+The following is an example of the commands for protection scheme migration from AES_GCM to AWS_KMS then back. Execute these commands at the `nifi` directory with the `nifi-toolkit` directory as a sibling directory. In addition, make sure to update `bootstrap-aws.conf` with your AWS KMS Key ARN/ID and have your credentials and region for AWS configured.

Review comment:
       ```suggestion
   The following is an example of the commands for protection scheme migration from AES_GCM to AWS_KMS then back. Execute these commands at the `nifi` directory with the `nifi-toolkit` directory as a sibling directory. In addition, make sure to update `bootstrap-aws.conf` with your AWS KMS Key ARN/ID and have your credentials and region configured.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r672456515



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);

Review comment:
       Making the changes

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            throw new SensitivePropertyProtectionException("Encrypt failed", e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r672456515



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);

Review comment:
       Making the changes

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            throw new SensitivePropertyProtectionException("Encrypt failed", e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669794639



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();

Review comment:
       Will remove the comments




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669794527



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();

Review comment:
       Will remove the comments




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r667180617



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";

Review comment:
       Yup, will make the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669784671



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");

Review comment:
       Making the change

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       Making the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r666954327



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java
##########
@@ -56,4 +56,11 @@
      * @return the raw value to be used by the application
      */
     String unprotect(String protectedValue) throws SensitivePropertyProtectionException;
+
+    /**
+     * Closes any clients that may have been opened by the SPP and releases
+     * any resources possibly used by any SPP implementation
+     * Note: If there is nothing to be done, then this function is a no-op
+     */
+    void close();

Review comment:
       What do you think about using slightly more generic language, like "Cleans up any resources allocated by the SPP", and naming this `cleanUp()`?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");

Review comment:
       If you move initialization to the constructor, I believe we'd want to do away with this warning.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";

Review comment:
       Rather than declaring these constants here, what do you think about referencing `PropertyProtectionScheme.AWS_KMS.getIdentifier()` and `getName()` where these are used below?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";

Review comment:
       Perhaps just AWS_PREFIX?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {

Review comment:
       Consider using final for all method parameters

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java
##########
@@ -257,4 +257,14 @@ public static int getMinCipherTextLength() {
     public static String getDelimiter() {
         return DELIMITER;
     }
+
+    /**
+     * Closes any clients that may have been opened by the SPP and releases
+     * any resources possibly used by any SPP implementation
+     * Note: If there is nothing to be done, then this function is a no-op

Review comment:
       Though I understand this is copied from the interface comments, perhaps we could provide an implementation-specific comment here instead, like "No cleanup necessary".

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;

Review comment:
       I believe these two can be final, since they're set in the constructor.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid

Review comment:
       are*

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf

Review comment:
       This is valuable documentation -- what do you think about adding the bit about "first attempts to use..." to the bootstrap-aws.conf file comments, so users will understand this behavior if they try configuring the SPP?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");

Review comment:
       It would be useful to include the above exception as a cause of this one.  Perhaps reword to "Valid credentials/configuration is required to initialize KMS client"

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AbstractHashiCorpVaultSensitivePropertyProvider.java
##########
@@ -129,4 +129,13 @@ public String getIdentifierKey() {
         return getProtectionScheme().getIdentifier(path);
     }
 
+    /**
+     * Closes any clients that may have been opened by the SPP and releases
+     * any resources possibly used by any SPP implementation
+     * Note: If there is nothing to be done, then this function is a no-op
+     */

Review comment:
       Same comment as above

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");

Review comment:
       Same comment regarding the exception cause

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);

Review comment:
       Consider using final for all local variables

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid
+        DescribeKeyResponse response = client.describeKey(request);
+        KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {

Review comment:
       Due to the way StandardSensitivePropertyProviderFactory is implemented, I don't believe bootstrapProperties can be null here (though it may be BootstrapProperties.EMPTY), so I don't think the null check adds value here.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {

Review comment:
       To encourage this SPP to be used only from the context of the SPP factory, let's make the constructor package scoped.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid
+        DescribeKeyResponse response = client.describeKey(request);
+        KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        logger.info("bootstrap-aws.conf file path" + ": " + filePath);

Review comment:
       Recommend removing this logger.info

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;

Review comment:
       I notice you initialize the client in `protect(...)`/`unprotect(...)` rather than in the constructor.  Can you provide some background on this choice?  Could we instead try to initialize the client in the constructor if the required aws properties are found to be configured?  Doing so would allow `client` to be declared as final.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {

Review comment:
       Consider using final for all exceptions

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid
+        DescribeKeyResponse response = client.describeKey(request);
+        KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");

Review comment:
       Perhaps `String.format("AWS KMS key [%s] is not enabled", keyId)`

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()

Review comment:
       Just to confirm, is the region also set when resolving the default credentials?

##########
File path: nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
##########
@@ -75,6 +75,13 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-mock</artifactId>
             <version>1.14.0-SNAPSHOT</version>
             <scope>test</scope>
+<!--The following is added to make sure the resolve dependencies issues bringing in AWS KMS dependency-->

Review comment:
       I think you need to remove "make sure the"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r672456515



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);

Review comment:
       Making the changes

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            throw new SensitivePropertyProtectionException("Encrypt failed", e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669794244



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r667180537



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java
##########
@@ -56,4 +56,11 @@
      * @return the raw value to be used by the application
      */
     String unprotect(String protectedValue) throws SensitivePropertyProtectionException;
+
+    /**
+     * Closes any clients that may have been opened by the SPP and releases
+     * any resources possibly used by any SPP implementation
+     * Note: If there is nothing to be done, then this function is a no-op
+     */
+    void close();

Review comment:
       I think cleanUp would be good as well, going to make the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669800473



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a null/empty cipher");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);
+            final byte[] plainBytes = decrypt(cipherBytes);
+            logger.debug(getName() + " decrypted a sensitive value successfully");

Review comment:
       Making the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669099360



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf
+# If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain

Review comment:
       Making the changes

##########
File path: nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669786483



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {

Review comment:
       Making the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669799880



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a null/empty cipher");

Review comment:
       Will make the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674365464



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured,
+# attempt to initialize credentials using default AWS credentials/configuration chain.
+
+# Optional AWS KMS Client Configuration/Credentials (all of them must be set in order to be used)

Review comment:
       Will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669796600



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {

Review comment:
       Will remove the function then




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669803749



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java
##########
@@ -56,4 +56,10 @@
      * @return the raw value to be used by the application
      */
     String unprotect(String protectedValue) throws SensitivePropertyProtectionException;
+
+    /**
+     * Cleans up resources that may have been allocated/used by an SPP implementation
+     * Note: If there is nothing to be done, then this function is a no-op (i.e. no cleanup necessary)

Review comment:
       Will remove this part of the comment




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668928890



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf

Review comment:
       I also noticed I made a small mixup in the documentation, will fix that as well




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674365558



##########
File path: nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured,
+# attempt to initialize credentials using default AWS credentials/configuration chain.
+
+# Optional AWS KMS Client Configuration/Credentials (all of them must be set in order to be used)

Review comment:
       Will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r667179843



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java
##########
@@ -257,4 +257,14 @@ public static int getMinCipherTextLength() {
     public static String getDelimiter() {
         return DELIMITER;
     }
+
+    /**
+     * Closes any clients that may have been opened by the SPP and releases
+     * any resources possibly used by any SPP implementation
+     * Note: If there is nothing to be done, then this function is a no-op

Review comment:
       I think that would be more descriptive yes, I will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669806075



##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service, or AWS KMS for short (https://aws.amazon.com/kms/) for encryption/decryption. All AWS KMS configuration/credentials details are to be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance. If the configuration/credentials details are not fully specified in `bootstrap-aws.conf`, then the protection scheme will attempt to use the default AWS credentials/configuration chain. Therefore, when using the AWS_KMS protection scheme, the `nifi(.registry)?.bootstrap.protection.aws.kms.conf` property in the `bootstrap.conf` specified using the `-b` flag must be available to the Encrypt Configuration Tool and must be configured as follows:
+
+===== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.kms.key.id`|The key id or ARN to be used by AWS KMS to identify the key used for encryption/decryption.|_none_

Review comment:
       Making the changes

##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service, or AWS KMS for short (https://aws.amazon.com/kms/) for encryption/decryption. All AWS KMS configuration/credentials details are to be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance. If the configuration/credentials details are not fully specified in `bootstrap-aws.conf`, then the protection scheme will attempt to use the default AWS credentials/configuration chain. Therefore, when using the AWS_KMS protection scheme, the `nifi(.registry)?.bootstrap.protection.aws.kms.conf` property in the `bootstrap.conf` specified using the `-b` flag must be available to the Encrypt Configuration Tool and must be configured as follows:
+
+===== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.kms.key.id`|The key id or ARN to be used by AWS KMS to identify the key used for encryption/decryption.|_none_
+|===
+
+===== Optional properties
+====== All of the following must be configured, or will be ignored entirely.
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.region`|The region to configure AWS KMS Client with for encryption/decryption.|_none_

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669789032



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);

Review comment:
       Making the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669804229



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
##########
@@ -105,6 +105,9 @@ public SensitivePropertyProvider getProvider(final PropertyProtectionScheme prot
         switch (protectionScheme) {
             case AES_GCM:
                 return providerMap.computeIfAbsent(protectionScheme, s -> new AESSensitivePropertyProvider(keyHex));
+            // Other providers may choose to pass getBootstrapProperties() into the constructor

Review comment:
       Will remove it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669797080



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668955791



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669797862



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");

Review comment:
       Will just remove it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669781688



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false

Review comment:
       I will just omit this then, the isSupported function calls another function that has the description of what to check. Making the changes!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669785906



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {

Review comment:
       RuntimeException is the more generic form of KmsException. Making this change and removing KmsException as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674364695



##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -699,6 +719,40 @@ for each phase (old vs. new), and any combination is sufficient:
 In order to change the protection scheme (e.g., migrating from AES encryption to Vault encryption), specify the `--protectionScheme`
 and `--oldProtectionScheme` in the migration command.
 
+The following is an example of the commands for protection scheme migration from AES_GCM to AWS_KMS then back. Execute these commands at the `nifi` directory with the `nifi-toolkit` directory as a sibling directory. In addition, make sure to update `bootstrap-aws.conf` with your AWS KMS Key ARN/ID and have your credentials and region configured.
+
+
+This command encrypts nifi.properties with the AES_GCM protection scheme
+----
+./../nifi-toolkit-*-SNAPSHOT/bin/encrypt-config.sh \
+-b conf/bootstrap.conf \
+-n conf/nifi.properties \
+-k 0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210 \
+-v
+----
+This command migrates nifi.properties from using AES_GCM to using AWS_KMS protection scheme
+----
+./../nifi-toolkit-*-SNAPSHOT/bin/encrypt-config.sh \
+-b conf/bootstrap.conf \
+-n conf/nifi.properties \
+-S AWS_KMS \
+-H AES_GCM \
+-e 0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210 \
+-m \
+-v
+----
+This command migrates nifi.properties back from AWS_KMS to AES_GCM protection scheme
+----
+./../nifi-toolkit-*-SNAPSHOT/bin/encrypt-config.sh \
+-b conf/bootstrap.conf \
+-n conf/nifi.properties \
+-S AES_GCM \
+-k 0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210 \
+-H AES_GCM \

Review comment:
       Will make the changes, although important thing to note for future changes of the encrypt-config tool is that this command will still produce the desired effect (i.e. changing AWS_KMS protection scheme back to AES_GCM protection scheme)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674364119



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.io.IOUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+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.key.id to the secret key id
+ * 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"
+ *
+ */
+
+public class AWSSensitivePropertyProviderIT {
+    private static final String SAMPLE_PLAINTEXT = "AWSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";

Review comment:
       Will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668956515



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668922321



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;

Review comment:
       I'm going to make awsBootstrapProperties final, but the way I'm setting keyId makes it that I couldn't make that variable final. I'm setting keyId the way it is set right now (i.e. in loadRequiredAWSProperties) for consistency with the other cloud providers SPP that I have, which sets more variables.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674339432



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.util.Base64;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";

Review comment:
       Recommend changing this property name to `aws.secret.key`:
   ```suggestion
       private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key";
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured,
+# attempt to initialize credentials using default AWS credentials/configuration chain.
+
+# Optional AWS KMS Client Configuration/Credentials (all of them must be set in order to be used)
+aws.access.key.id=
+aws.secret.key.id=

Review comment:
       ```suggestion
   aws.secret.key=
   ```

##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service (https://aws.amazon.com/kms/) 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.
+
+===== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.kms.key.id`|The identifier or ARN that the AWS KMS client uses for encryption and decryption.|_none_
+|===
+
+===== 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 KMS Client.|_none_
+|`aws.access.key.id`|The access key ID credential used to access AWS KMS.|_none_
+|`aws.secret.key.id`|The secret key ID credential used to access AWS KMS.|_none_

Review comment:
       ```suggestion
   |`aws.secret.key`|The secret key credential used to access AWS KMS.|_none_
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.io.IOUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+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.key.id to the secret key id

Review comment:
       Should be changed to `aws.secret.key`:
   ```suggestion
    * set the system property -Daws.secret.key to the secret key
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.io.IOUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+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.key.id to the secret key id
+ * 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"
+ *
+ */
+
+public class AWSSensitivePropertyProviderIT {
+    private static final String SAMPLE_PLAINTEXT = "AWSSensitivePropertyProviderIT SAMPLE-PLAINTEXT";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";

Review comment:
       ```suggestion
       private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key";
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured,
+# attempt to initialize credentials using default AWS credentials/configuration chain.
+
+# Optional AWS KMS Client Configuration/Credentials (all of them must be set in order to be used)

Review comment:
       Recommend rewording to clarify the implementation:
   ```suggestion
   # NiFi uses the following properties when authentication to AWS when all values are provided.
   # NiFi uses the default AWS credentials provider chain when one or more or the following properties are blank
   # AWS SDK documentation describes the default credential retrieval order:
   # https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html#credentials-chain
   ```

##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -699,6 +719,40 @@ for each phase (old vs. new), and any combination is sufficient:
 In order to change the protection scheme (e.g., migrating from AES encryption to Vault encryption), specify the `--protectionScheme`
 and `--oldProtectionScheme` in the migration command.
 
+The following is an example of the commands for protection scheme migration from AES_GCM to AWS_KMS then back. Execute these commands at the `nifi` directory with the `nifi-toolkit` directory as a sibling directory. In addition, make sure to update `bootstrap-aws.conf` with your AWS KMS Key ARN/ID and have your credentials and region configured.
+
+
+This command encrypts nifi.properties with the AES_GCM protection scheme
+----
+./../nifi-toolkit-*-SNAPSHOT/bin/encrypt-config.sh \
+-b conf/bootstrap.conf \
+-n conf/nifi.properties \
+-k 0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210 \
+-v
+----
+This command migrates nifi.properties from using AES_GCM to using AWS_KMS protection scheme
+----
+./../nifi-toolkit-*-SNAPSHOT/bin/encrypt-config.sh \
+-b conf/bootstrap.conf \
+-n conf/nifi.properties \
+-S AWS_KMS \
+-H AES_GCM \
+-e 0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210 \
+-m \
+-v
+----
+This command migrates nifi.properties back from AWS_KMS to AES_GCM protection scheme
+----
+./../nifi-toolkit-*-SNAPSHOT/bin/encrypt-config.sh \
+-b conf/bootstrap.conf \
+-n conf/nifi.properties \
+-S AES_GCM \
+-k 0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210 \
+-H AES_GCM \

Review comment:
       Based on the description, it looks like the previous scheme should be AWS_KMS for the example command.
   ```suggestion
   -H AWS_KMS \
   ```

##########
File path: nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured,
+# attempt to initialize credentials using default AWS credentials/configuration chain.
+
+# Optional AWS KMS Client Configuration/Credentials (all of them must be set in order to be used)

Review comment:
       Recommend rewording to clarify the implementation:
   ```suggestion
   # NiFi uses the following properties when authentication to AWS when all values are provided.
   # NiFi uses the default AWS credentials provider chain when one or more or the following properties are blank
   # AWS SDK documentation describes the default credential retrieval order:
   # https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html#credentials-chain
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669099120



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use credentials/configuration in bootstrap-aws.conf
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain
+     * Note: This does not verify if credentials are valid
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return True if bootstrap-aws.conf contains the required properties for AWS SPP, False otherwise
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value
+     * @return the value to persist in the {@code nifi.properties} file
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }

Review comment:
       Making the changes! Moving this to another checkAndInitializeClient() function.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
gresockj commented on pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#issuecomment-877382128


   By the way, I also tested this by encrypting nifi.properties with the encrypt-tool and then decrypting by running NiFi.  Works as designed!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r667191074



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {

Review comment:
       Currently I have the AWS SPP in its own 'aws' package, and making it package scoped would make it inaccessible from the SPP factory. To implement this I would need to take it out of that package.
   
   I think this change would be good though to encourage use from just the SPP factory, so I am proceeding ahead to make these changes. 

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid
+        DescribeKeyResponse response = client.describeKey(request);
+        KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {

Review comment:
       Will make the change

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid
+        DescribeKeyResponse response = client.describeKey(request);
+        KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        logger.info("bootstrap-aws.conf file path" + ": " + filePath);

Review comment:
       Will do

##########
File path: nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
##########
@@ -75,6 +75,13 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-mock</artifactId>
             <version>1.14.0-SNAPSHOT</version>
             <scope>test</scope>
+<!--The following is added to make sure the resolve dependencies issues bringing in AWS KMS dependency-->

Review comment:
       Will remove this




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668949375



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()

Review comment:
       The resolveCredentials is there as means to check the credentials. Checking the region happens during the client building based on my testing for the default AWS credentials/configuration chain.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669787168



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException

Review comment:
       Removing this comment




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669804912



##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service, or AWS KMS for short (https://aws.amazon.com/kms/) for encryption/decryption. All AWS KMS configuration/credentials details are to be stored in the `bootstrap-aws.conf` file, as referenced in the `bootstrap.conf` of a NiFi or NiFi Registry instance. If the configuration/credentials details are not fully specified in `bootstrap-aws.conf`, then the protection scheme will attempt to use the default AWS credentials/configuration chain. Therefore, when using the AWS_KMS protection scheme, the `nifi(.registry)?.bootstrap.protection.aws.kms.conf` property in the `bootstrap.conf` specified using the `-b` flag must be available to the Encrypt Configuration Tool and must be configured as follows:

Review comment:
       Will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668922798



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf

Review comment:
       I think thats a great idea to have this documentation in the bootstrap-aws.conf as well. Making the changes!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r667180900



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";

Review comment:
       I think that would be better, I originally had it this way because I referenced it more than once, making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669807702



##########
File path: nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/pom.xml
##########
@@ -75,6 +75,13 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-mock</artifactId>
             <version>1.14.0-SNAPSHOT</version>
             <scope>test</scope>
+<!--The following is added to resolve dependencies issues bringing in AWS KMS dependency-->

Review comment:
       Adjusting the comment




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r667180985



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid
+        DescribeKeyResponse response = client.describeKey(request);
+        KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668915742



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {

Review comment:
       Noted and making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669783200



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {

Review comment:
       Making the change!

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");

Review comment:
       Making the change!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674354421



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.internal.util.io.IOUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+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.key.id to the secret key id

Review comment:
       Will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669807125



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/main/java/org/apache/nifi/properties/NiFiPropertiesLoader.java
##########
@@ -173,8 +173,14 @@ public NiFiProperties load(final File file) {
                     .getSupportedSensitivePropertyProviders()
                     .forEach(protectedNiFiProperties::addSensitivePropertyProvider);
         }
-
-        return protectedNiFiProperties.getUnprotectedProperties();
+        NiFiProperties props = protectedNiFiProperties.getUnprotectedProperties();
+        if (protectedNiFiProperties.hasProtectedKeys()) {
+            // releases resources used by SPP

Review comment:
       Removing the comment




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r667215090



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                logger.error("Default credentials/configuration for AWS are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            logger.error("The AWS KMS Client failed to open, cannot validate key");
+            throw new SensitivePropertyProtectionException("The AWS KMS Client failed to open, cannot validate key");
+        }
+        if (StringUtils.isBlank(keyId)) {
+            logger.error("The AWS KMS Key provided is blank");
+            throw new SensitivePropertyProtectionException("The AWS KMS Key provided is blank");
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key is valid
+        DescribeKeyResponse response = client.describeKey(request);
+        KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            throw new SensitivePropertyProtectionException("The key is not enabled");
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {

Review comment:
       Actually this reminds me of a case using encrypt-config tool. My setup was that I have nifi and nifi-toolkit as sibling directories. And if I run the encrypt-config tool from the nifi-toolkit directory, the bootstrapProperties will be null.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669798868



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669780912



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java
##########
@@ -257,4 +257,12 @@ public static int getMinCipherTextLength() {
     public static String getDelimiter() {
         return DELIMITER;
     }
+
+    /**
+     * No cleanup necessary
+     */
+    @Override
+    public void cleanUp() {
+        return;

Review comment:
       Making the changes!

##########
File path: nifi-assembly/NOTICE
##########
@@ -1013,6 +1013,13 @@ The following binary components are provided under the Apache Software License v
       Since product implements StAX API, it has dependencies to StAX API
       classes.
 
+  (ASLv2) AWS SDK for Java 2.0
+      The following NOTICE information applies:
+        Copyright 2010-2019 Amazon.com, Inc. or its affiliates. All Rights Reserved.

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669786171



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       Making the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669789260



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);

Review comment:
       Making the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r671526977



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);

Review comment:
       The Bouncy Castle `Base64` class can be replaced with the standard Java `Base64.Encoder` class.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AbstractHashiCorpVaultSensitivePropertyProvider.java
##########
@@ -129,4 +129,11 @@ public String getIdentifierKey() {
         return getProtectionScheme().getIdentifier(path);
     }
 
+    /**
+     * No cleanup necessary
+     */
+    @Override
+    public void cleanUp() {
+        return;

Review comment:
       This `return` can be removed.

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        Objects.requireNonNull(bootstrapProperties, "The file bootstrap.conf provided to AWS SPP is null");
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        loadRequiredAWSProperties(awsBootstrapProperties);
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("AWS Bootstrap Properties are required for KMS Client initialization");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNoneBlank(accessKeyId, secretKeyId, region)) {
+            logger.debug("Using AWS credentials from bootstrap properties");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final RuntimeException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            logger.debug("Using AWS credentials from default credentials provider");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (client == null) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("AWS KMS properties file path not configured in bootstrap properties");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        final EncryptResponse response = client.encrypt(encryptRequest);
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        final DecryptResponse response = client.decrypt(decryptRequest);
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (client == null) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            throw new SensitivePropertyProtectionException("Encrypt failed", e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a blank value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);

Review comment:
       Replacing this Bouncy Castle `Base64` with Java `Base64.Encoder` will also remove the need to catch the Bouncy Castle `DecoderException`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669783961



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");

Review comment:
       Making the change!




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma closed pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma closed pull request #5202:
URL: https://github.com/apache/nifi/pull/5202


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669797245



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] gresockj commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
gresockj commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668964879



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf
+# If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain

Review comment:
       Minor formatting comment -- could you limit the line length to about half this?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use credentials/configuration in bootstrap-aws.conf
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain
+     * Note: This does not verify if credentials are valid
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (IOException e) {

Review comment:
       final*

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use credentials/configuration in bootstrap-aws.conf
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain
+     * Note: This does not verify if credentials are valid
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open
+     * @return true if the client has been initialized and open, false otherwise
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present
+     * Note: This function does not verify if the key is correctly formatted/valid
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return True if bootstrap-aws.conf contains the required properties for AWS SPP, False otherwise
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value
+     * @return the value to persist in the {@code nifi.properties} file
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }

Review comment:
       I think you can define a method to reuse this code in unprotect as well.

##########
File path: nifi-registry/nifi-registry-core/nifi-registry-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured, attempt to initialize credentials using default AWS credentials/configuration chain

Review comment:
       Same comment as above




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668934723



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");

Review comment:
       I'll keep this warning since I'm keeping initialization of the client in protect/unprotect. I would make the message a bit more descriptive.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669790392



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }

Review comment:
       I will remove the props != null check in the calling method, and keep this method. I'll keep it for consistency with other cloud provider SPPs where they load more things from the bootstrapproperties




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669788160



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {

Review comment:
       I will change the naming, I think this function makes it neater and the use might be expanded upon. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669806838



##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -699,6 +719,39 @@ for each phase (old vs. new), and any combination is sufficient:
 In order to change the protection scheme (e.g., migrating from AES encryption to Vault encryption), specify the `--protectionScheme`
 and `--oldProtectionScheme` in the migration command.
 
+The following is an example of the commands for protection scheme migration from AES_GCM to AWS_KMS then back. Execute these commands at the `nifi` directory with the `nifi-toolkit` directory as a sibling directory. In addition, make sure to update `bootstrap-aws.conf` with your AWS KMS Key ARN/ID and have your credentials and region for AWS configured.

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669793563



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668919045



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;

Review comment:
       I did this in order to try and make the AWS Sensitive Property Provider initialize a few things as possible if we have everything configured, but we don't actually use the sensitive property provider. 
   
   For example, in the Standard SPP Factory, the getSupportedSensitivePropertyProviders() function would see that AWS SPP is supported if the bootstrap-aws.conf and AWS credentials and configurations are present and properly configured, however we still might not use the AWS SPP for protection and unprotection.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669784757



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       Making the change, also made the change to the corresponding logger in the "else" statement"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674354289



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.util.Base64;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+import java.util.Objects;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";

Review comment:
       Will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674365041



##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/bootstrap-aws.conf
##########
@@ -0,0 +1,28 @@
+#
+# 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.
+#
+
+# AWS KMS Key ID is required to be configured for AWS KMS Sensitive Property Provider
+aws.kms.key.id=
+
+# First attempts to use credentials/configuration in bootstrap-aws.conf.
+# If credentials/configuration in bootstrap-aws.conf is not fully configured,
+# attempt to initialize credentials using default AWS credentials/configuration chain.
+
+# Optional AWS KMS Client Configuration/Credentials (all of them must be set in order to be used)
+aws.access.key.id=
+aws.secret.key.id=

Review comment:
       Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r674364245



##########
File path: nifi-docs/src/main/asciidoc/toolkit-guide.adoc
##########
@@ -504,6 +504,26 @@ This protection scheme uses HashiCorp Vault's Transit Secrets Engine (https://ww
 |`vault.ssl.trust-store-password`|Truststore password.  Required if the Vault server is TLS-enabled|_none_
 |===
 
+==== AWS_KMS
+This protection scheme uses AWS Key Management Service (https://aws.amazon.com/kms/) 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.
+
+===== Required properties
+[options="header,footer"]
+|===
+|Property Name|Description|Default
+|`aws.kms.key.id`|The identifier or ARN that the AWS KMS client uses for encryption and decryption.|_none_
+|===
+
+===== 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 KMS Client.|_none_
+|`aws.access.key.id`|The access key ID credential used to access AWS KMS.|_none_
+|`aws.secret.key.id`|The secret key ID credential used to access AWS KMS.|_none_

Review comment:
       Will make the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r669800300



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String AWS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private final BootstrapProperties awsBootstrapProperties;
+    private KmsClient client;
+    private String keyId;
+
+
+    AWSSensitivePropertyProvider(final BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS.
+     * First attempts to use credentials/configuration in bootstrap-aws.conf.
+     * If credentials/configuration in bootstrap-aws.conf is not fully configured,
+     * attempt to initialize credentials using default AWS credentials/configuration chain.
+     * Note: This does not verify if credentials are valid.
+     */
+    private final void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        final String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        final String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        final String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                final AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (final KmsException | NullPointerException | IllegalArgumentException e) {
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                final DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()
+                        .credentialsProvider(credentialsProvider)
+                        .build();
+            } catch (final SdkClientException e) {
+                // this exception occurs if default credentials are not provided
+                final String msg = "Valid configuration/credentials are required to initialize KMS client";
+                logger.error(msg);
+                throw new SensitivePropertyProtectionException(msg, e);
+            }
+        }
+    }
+
+    /**
+     * Checks if the client used to communicate with AWS KMS service is open.
+     * @return true if the client has been initialized and open, false otherwise.
+     */
+    private final boolean isClientOpen() {
+        return client != null;
+    }
+
+    /**
+     * Validates the key ARN, credentials and configuration provided by the user.
+     * Note: This function performs checks on the key and indirectly also validates the credentials and
+     * configurations provided during the initialization of the client.
+     */
+    private final void validate() throws KmsException, SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            final String msg = "The AWS KMS Client failed to open, cannot validate key";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+        if (StringUtils.isBlank(keyId)) {
+            final String msg = "The AWS KMS key provided is blank";
+            logger.error(msg);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+
+        // asking for a Key Description is the best way to check whether a key is valid
+        // because AWS KMS accepts various formats for its keys.
+
+        final DescribeKeyRequest request = DescribeKeyRequest.builder()
+                .keyId(keyId)
+                .build();
+
+        // using the KmsClient in a DescribeKey request indirectly also verifies if the credentials provided
+        // during the initialization of the key are valid
+        final DescribeKeyResponse response = client.describeKey(request);
+        final KeyMetadata metadata = response.keyMetadata();
+
+        if (!metadata.enabled()) {
+            final String msg = String.format("AWS KMS key [%s] is not enabled", keyId);
+            throw new SensitivePropertyProtectionException(msg);
+        }
+    }
+
+    /**
+     * Checks if we have a key ID from AWS KMS and loads it into {@link #keyId}. Will load null if key is not present.
+     * Note: This function does not verify if the key is correctly formatted/valid.
+     * @param props the properties representing bootstrap-aws.conf
+     */
+    private void loadRequiredAWSProperties(final BootstrapProperties props) {
+        if (props != null) {
+            keyId = props.getProperty(KMS_KEY_PROPS_NAME);
+        }
+    }
+
+
+    /**
+     * Checks bootstrap.conf to check if BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF property is
+     * configured to the bootstrap-aws.conf file. Also will try to load bootstrap-aws.conf to {@link #awsBootstrapProperties}.
+     * @param bootstrapProperties BootstrapProperties object corresponding to bootstrap.conf.
+     * @return BootstrapProperties object corresponding to bootstrap-aws.conf, null otherwise.
+     */
+    private BootstrapProperties getAWSBootstrapProperties(final BootstrapProperties bootstrapProperties) {
+        if (bootstrapProperties == null) {
+            logger.warn("The file bootstrap.conf provided to AWS SPP is null");
+            return null;
+        }
+
+        final BootstrapProperties cloudBootstrapProperties;
+
+        // Load the bootstrap-aws.conf file based on path specified in
+        // "nifi.bootstrap.protection.aws.kms.conf" property of bootstrap.conf
+        final String filePath = bootstrapProperties.getProperty(BootstrapPropertyKey.AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF).orElse(null);
+        if (StringUtils.isBlank(filePath)) {
+            logger.warn("File Path to bootstrap-aws.conf in bootstrap.conf is blank");
+            return null;
+        }
+
+        try {
+            cloudBootstrapProperties = AbstractBootstrapPropertiesLoader.loadBootstrapProperties(
+                    Paths.get(filePath), AWS_PREFIX);
+        } catch (final IOException e) {
+            throw new SensitivePropertyProtectionException("Could not load " + filePath, e);
+        }
+
+        return cloudBootstrapProperties;
+    }
+
+    /**
+     * Checks bootstrap-aws.conf for the required configurations for AWS KMS encrypt/decrypt operations.
+     * Note: This does not check for credentials/region configurations.
+     *       Credentials/configuration will be checked during the first protect/unprotect call during runtime.
+     * @return true if bootstrap-aws.conf contains the required properties for AWS SPP, false otherwise.
+     */
+    private boolean hasRequiredAWSProperties() {
+        return awsBootstrapProperties != null && StringUtils.isNotBlank(keyId);
+    }
+
+    @Override
+    public boolean isSupported() {
+        return hasRequiredAWSProperties();
+    }
+
+    @Override
+    protected PropertyProtectionScheme getProtectionScheme() {
+        return PropertyProtectionScheme.AWS_KMS;
+    }
+
+    /**
+     * Returns the name of the underlying implementation.
+     *
+     * @return the name of this sensitive property provider.
+     */
+    @Override
+    public String getName() {
+        return PropertyProtectionScheme.AWS_KMS.getName();
+    }
+
+    /**
+     * Returns the key used to identify the provider implementation in {@code nifi.properties}.
+     *
+     * @return the key to persist in the sibling property.
+     */
+    @Override
+    public String getIdentifierKey() {
+        return PropertyProtectionScheme.AWS_KMS.getIdentifier();
+    }
+
+
+    /**
+     * Returns the ciphertext blob of this value encrypted using an AWS KMS CMK.
+     *
+     * @return the ciphertext blob to persist in the {@code nifi.properties} file.
+     */
+    private byte[] encrypt(final byte[] input) {
+        final SdkBytes plainBytes = SdkBytes.fromByteArray(input);
+
+        // builds an encryption request to be sent to the kmsClient
+        final EncryptRequest encryptRequest = EncryptRequest.builder()
+                .keyId(keyId)
+                .plaintext(plainBytes)
+                .build();
+
+        // sends request, records response
+        final EncryptResponse response = client.encrypt(encryptRequest);
+
+        // get encrypted data
+        final SdkBytes encryptedData = response.ciphertextBlob();
+
+        return encryptedData.asByteArray();
+    }
+
+    /**
+     * Returns the value corresponding to a ciphertext blob decrypted using an AWS KMS CMK.
+     *
+     * @return the "unprotected" byte[] of this value, which could be used by the application.
+     */
+    private byte[] decrypt(final byte[] input) {
+        final SdkBytes cipherBytes = SdkBytes.fromByteArray(input);
+
+        // builds a decryption request to be sent to the kmsClient
+        final DecryptRequest decryptRequest = DecryptRequest.builder()
+                .ciphertextBlob(cipherBytes)
+                .keyId(keyId)
+                .build();
+
+        // sends request, records response
+        final DecryptResponse response = client.decrypt(decryptRequest);
+
+        // get decrypted data
+        final SdkBytes decryptedData = response.plaintext();
+
+        return decryptedData.asByteArray();
+    }
+
+    /**
+     * Checks if the client is open and if not, initializes the client and validates the key required for AWS KMS.
+     */
+    private void checkAndInitializeClient() throws SensitivePropertyProtectionException {
+        if (!isClientOpen()) {
+            try {
+                initializeClient();
+                validate();
+            } catch (final SdkClientException | KmsException | SensitivePropertyProtectionException e) {
+                logger.error("Encountered an error initializing the client for {}: {}", getName(), e.getMessage());
+                throw new SensitivePropertyProtectionException("Error initializing the AWS KMS Client", e);
+            }
+        }
+    }
+
+    /**
+     * Returns the "protected" form of this value. This is a form which can safely be persisted in the {@code nifi.properties} file without compromising the value.
+     * An encryption-based provider would return a cipher text, while a remote-lookup provider could return a unique ID to retrieve the secured value.
+     *
+     * @param unprotectedValue the sensitive value.
+     * @return the value to persist in the {@code nifi.properties} file.
+     */
+    @Override
+    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(unprotectedValue)) {
+            throw new IllegalArgumentException("Cannot encrypt a null/empty value");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] plainBytes = unprotectedValue.getBytes(PROPERTY_CHARSET);
+            final byte[] cipherBytes = encrypt(plainBytes);
+            logger.debug(getName() + " encrypted a sensitive value successfully");
+            return Base64.toBase64String(cipherBytes);
+        } catch (final SdkClientException | KmsException | EncoderException e) {
+            final String msg = "Error encrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);
+        }
+    }
+
+    /**
+     * Returns the "unprotected" form of this value. This is the raw sensitive value which is used by the application logic.
+     * An encryption-based provider would decrypt a cipher text and return the plaintext, while a remote-lookup provider could retrieve the secured value.
+     *
+     * @param protectedValue the protected value read from the {@code nifi.properties} file.
+     * @return the raw value to be used by the application.
+     */
+    @Override
+    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+        if (StringUtils.isBlank(protectedValue)) {
+            throw new IllegalArgumentException("Cannot decrypt a null/empty cipher");
+        }
+
+        checkAndInitializeClient();
+
+        try {
+            final byte[] cipherBytes = Base64.decode(protectedValue);
+            final byte[] plainBytes = decrypt(cipherBytes);
+            logger.debug(getName() + " decrypted a sensitive value successfully");
+            return new String(plainBytes, PROPERTY_CHARSET);
+        } catch (final SdkClientException | KmsException | DecoderException e) {
+            final String msg = "Error decrypting a protected value";
+            logger.error(msg, e);
+            throw new SensitivePropertyProtectionException(msg, e);

Review comment:
       Making the change




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668949375



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {
+                logger.error("Credentials/Configuration provided in bootstrap-aws.conf are invalid");
+                throw new SensitivePropertyProtectionException("Require valid credentials/configuration to initialize KMS client");
+            }
+        } else {
+            // attempts to initialize client with credentials provider chain
+            logger.debug("Credentials/Configuration not provided in bootstrap-aws.conf, attempting to use default configuration");
+            try {
+                DefaultCredentialsProvider credentialsProvider = DefaultCredentialsProvider.builder()
+                        .build();
+                // the following is needed to check the default credential builder, if it fails, throws SdkClientException
+                credentialsProvider.resolveCredentials();
+                client = KmsClient.builder()

Review comment:
       The resolveCredentials is there as means to check the credentials. Checking the region happens during the client building based on my testing.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] emiliosetiadarma commented on a change in pull request #5202: NIFI-6325 Added AWS Sensitive Property Provider

Posted by GitBox <gi...@apache.org>.
emiliosetiadarma commented on a change in pull request #5202:
URL: https://github.com/apache/nifi/pull/5202#discussion_r668947970



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);

Review comment:
       Will do! Making the changes

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/aws/AWSSensitivePropertyProvider.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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.aws;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.AbstractSensitivePropertyProvider;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.PropertyProtectionScheme;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
+
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.DecoderException;
+import org.bouncycastle.util.encoders.EncoderException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kms.KmsClient;
+import software.amazon.awssdk.services.kms.model.DecryptRequest;
+import software.amazon.awssdk.services.kms.model.DecryptResponse;
+import software.amazon.awssdk.services.kms.model.DescribeKeyRequest;
+import software.amazon.awssdk.services.kms.model.DescribeKeyResponse;
+import software.amazon.awssdk.services.kms.model.EncryptRequest;
+import software.amazon.awssdk.services.kms.model.EncryptResponse;
+import software.amazon.awssdk.services.kms.model.KeyMetadata;
+import software.amazon.awssdk.services.kms.model.KmsException;
+
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
+
+public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private static final Logger logger = LoggerFactory.getLogger(AWSSensitivePropertyProvider.class);
+
+    private static final String IMPLEMENTATION_NAME = "AWS KMS Sensitive Property Provider";
+    private static final String IMPLEMENTATION_KEY = "aws/kms";
+
+    private static final String AWS_KMS_PREFIX = "aws";
+    private static final String ACCESS_KEY_PROPS_NAME = "aws.access.key.id";
+    private static final String SECRET_KEY_PROPS_NAME = "aws.secret.key.id";
+    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 Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    private KmsClient client;
+    private BootstrapProperties awsBootstrapProperties;
+    private String keyId;
+
+    public AWSSensitivePropertyProvider(BootstrapProperties bootstrapProperties) throws SensitivePropertyProtectionException {
+        super(bootstrapProperties);
+        // if either awsBootstrapProperties or keyId is loaded as null values, then isSupported will return false
+        awsBootstrapProperties = getAWSBootstrapProperties(bootstrapProperties);
+        if (awsBootstrapProperties != null) {
+            loadRequiredAWSProperties(awsBootstrapProperties);
+        }
+    }
+
+    /**
+     * Initializes the KMS Client to be used for encrypt, decrypt and other interactions with AWS KMS
+     * First attempts to use default AWS Credentials Provider Chain
+     * If that attempt fails, attempt to initialize credentials using bootstrap-aws.conf
+     * Note: This does not verify if credentials are valid
+     */
+    private void initializeClient() {
+        if (awsBootstrapProperties == null) {
+            logger.warn("Cannot initialize client if awsBootstrapProperties is null");
+            return;
+        }
+        String accessKeyId = awsBootstrapProperties.getProperty(ACCESS_KEY_PROPS_NAME);
+        String secretKeyId = awsBootstrapProperties.getProperty(SECRET_KEY_PROPS_NAME);
+        String region = awsBootstrapProperties.getProperty(REGION_KEY_PROPS_NAME);
+
+        if (StringUtils.isNotBlank(accessKeyId) && StringUtils.isNotBlank(secretKeyId) && StringUtils.isNotBlank(region)) {
+            logger.debug("Credentials/Configuration provided in bootstrap-aws.conf");
+            try {
+                AwsBasicCredentials credentials = AwsBasicCredentials.create(accessKeyId, secretKeyId);
+                client = KmsClient.builder()
+                        .region(Region.of(region))
+                        .credentialsProvider(StaticCredentialsProvider.create(credentials))
+                        .build();
+            } catch (KmsException | NullPointerException | IllegalArgumentException e) {

Review comment:
       Will do! Making the changes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org