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/09/22 17:22:15 UTC

[GitHub] [nifi] gresockj opened a new pull request #5410: NIFI-9221: Add AWS SecretsManager Sensitive Props Provider

gresockj opened a new pull request #5410:
URL: https://github.com/apache/nifi/pull/5410


   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Adds a Sensitive Property Provider that stores properties as AWS Secrets Manager secrets_
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] 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.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] 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:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] 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:
   - [ ] 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] exceptionfactory commented on a change in pull request #5410: NIFI-9221: Add AWS SecretsManager Sensitive Props Provider

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



##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.secretsmanager.model.SecretsManagerException;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class AwsSecretsManagerSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private final SecretsManagerClient client;
+    private final ObjectMapper objectMapper;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    AwsSecretsManagerSensitivePropertyProvider(final SecretsManagerClient client) {
+        super(null);
+
+        this.client = client;
+        this.objectMapper = new ObjectMapper();
+    }
+
+    @Override
+    public boolean isSupported() {
+        return client != null;
+    }
+
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+        Objects.requireNonNull(unprotectedValue, "Property value must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+
+        try {
+            writeLock.lock();
+            final String secretName = context.getContextName();
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            final ObjectNode secretObject = (ObjectNode) secretKeyValuesOptional.orElse(objectMapper.createObjectNode());
+
+            secretObject.put(context.getPropertyName(), unprotectedValue);
+            final String secretString = objectMapper.writeValueAsString(secretObject);
+
+            if (secretKeyValuesOptional.isPresent()) {
+                client.putSecretValue(builder -> builder.secretId(secretName).secretString(secretString));
+            } else {
+                client.createSecret(builder -> builder.name(secretName).secretString(secretString));
+            }
+            return context.getContextKey();
+        } catch (final SecretsManagerException | JsonProcessingException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Stored for [%s]", context), e);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+        try {
+            readLock.lock();
+
+            String propertyValue = null;
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            if (secretKeyValuesOptional.isPresent()) {
+                final JsonNode secretKeyValues = secretKeyValuesOptional.get();
+                final String propertyName = context.getPropertyName();
+                if (secretKeyValues.has(propertyName)) {
+                    propertyValue = secretKeyValues.get(propertyName).textValue();
+                }
+            }
+            if (propertyValue == null) {
+                throw new SensitivePropertyProtectionException(String.format("Property [%s] Not Found in AWS Secrets Manager Secret", context.getContextKey()));
+            }
+
+            return propertyValue;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    /**
+     * Returns the optional parsed JSON from the matching secret, or empty if the secret does not exist.
+     * @param context The property context
+     * @return The optional parsed JSON, or empty if the secret does not exist
+     */
+    private Optional<JsonNode> getSecretKeyValues(final ProtectedPropertyContext context) {
+        try {
+            final GetSecretValueResponse response = client.getSecretValue(builder -> builder.secretId(context.getContextName()));
+
+            if (response.secretString() == null) {
+                throw new SensitivePropertyProtectionException(String.format("Found No Secret String in AWS Secrets Manager Secret for [%s]",

Review comment:
       Suggest rewording:
   ```suggestion
                   throw new SensitivePropertyProtectionException(String.format("AWS Secret Name [%s] string value not found",
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.secretsmanager.model.SecretsManagerException;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class AwsSecretsManagerSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private final SecretsManagerClient client;
+    private final ObjectMapper objectMapper;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    AwsSecretsManagerSensitivePropertyProvider(final SecretsManagerClient client) {
+        super(null);
+
+        this.client = client;
+        this.objectMapper = new ObjectMapper();
+    }
+
+    @Override
+    public boolean isSupported() {
+        return client != null;
+    }
+
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+        Objects.requireNonNull(unprotectedValue, "Property value must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+
+        try {
+            writeLock.lock();
+            final String secretName = context.getContextName();
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            final ObjectNode secretObject = (ObjectNode) secretKeyValuesOptional.orElse(objectMapper.createObjectNode());
+
+            secretObject.put(context.getPropertyName(), unprotectedValue);
+            final String secretString = objectMapper.writeValueAsString(secretObject);
+
+            if (secretKeyValuesOptional.isPresent()) {
+                client.putSecretValue(builder -> builder.secretId(secretName).secretString(secretString));
+            } else {
+                client.createSecret(builder -> builder.name(secretName).secretString(secretString));
+            }
+            return context.getContextKey();
+        } catch (final SecretsManagerException | JsonProcessingException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Stored for [%s]", context), e);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+        try {
+            readLock.lock();
+
+            String propertyValue = null;
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            if (secretKeyValuesOptional.isPresent()) {
+                final JsonNode secretKeyValues = secretKeyValuesOptional.get();
+                final String propertyName = context.getPropertyName();
+                if (secretKeyValues.has(propertyName)) {
+                    propertyValue = secretKeyValues.get(propertyName).textValue();
+                }
+            }
+            if (propertyValue == null) {
+                throw new SensitivePropertyProtectionException(String.format("Property [%s] Not Found in AWS Secrets Manager Secret", context.getContextKey()));

Review comment:
       What do you think about updating this message to separate the Secret Name and the Property Name?
   ```suggestion
                   throw new SensitivePropertyProtectionException(String.format("AWS Secret Name [%s] Property Name [%s] not found", context.getContextName(), context.getPropertyName()));
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/configuration/AbstractAwsClientProvider.java
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties.configuration;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.SensitivePropertyProtectionException;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
+import software.amazon.awssdk.core.SdkClient;
+
+import java.util.Properties;
+
+/**
+ * Amazon Web Services Service Client Provider base class
+ */
+public abstract class AbstractAwsClientProvider<Client extends SdkClient> extends BootstrapPropertiesClientProvider<Client> {

Review comment:
       Recommend changing `Client` to `T` following general conventions for generics:
   ```suggestion
   public abstract class AbstractAwsClientProvider<T extends SdkClient> extends BootstrapPropertiesClientProvider<T> {
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.secretsmanager.model.SecretsManagerException;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class AwsSecretsManagerSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private final SecretsManagerClient client;
+    private final ObjectMapper objectMapper;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    AwsSecretsManagerSensitivePropertyProvider(final SecretsManagerClient client) {
+        super(null);
+
+        this.client = client;
+        this.objectMapper = new ObjectMapper();
+    }
+
+    @Override
+    public boolean isSupported() {
+        return client != null;
+    }
+
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+        Objects.requireNonNull(unprotectedValue, "Property value must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+
+        try {
+            writeLock.lock();
+            final String secretName = context.getContextName();
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            final ObjectNode secretObject = (ObjectNode) secretKeyValuesOptional.orElse(objectMapper.createObjectNode());
+
+            secretObject.put(context.getPropertyName(), unprotectedValue);
+            final String secretString = objectMapper.writeValueAsString(secretObject);
+
+            if (secretKeyValuesOptional.isPresent()) {
+                client.putSecretValue(builder -> builder.secretId(secretName).secretString(secretString));
+            } else {
+                client.createSecret(builder -> builder.name(secretName).secretString(secretString));
+            }
+            return context.getContextKey();
+        } catch (final SecretsManagerException | JsonProcessingException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Stored for [%s]", context), e);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+        try {
+            readLock.lock();
+
+            String propertyValue = null;
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            if (secretKeyValuesOptional.isPresent()) {
+                final JsonNode secretKeyValues = secretKeyValuesOptional.get();
+                final String propertyName = context.getPropertyName();
+                if (secretKeyValues.has(propertyName)) {
+                    propertyValue = secretKeyValues.get(propertyName).textValue();
+                }
+            }
+            if (propertyValue == null) {
+                throw new SensitivePropertyProtectionException(String.format("Property [%s] Not Found in AWS Secrets Manager Secret", context.getContextKey()));
+            }
+
+            return propertyValue;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    /**
+     * Returns the optional parsed JSON from the matching secret, or empty if the secret does not exist.
+     * @param context The property context
+     * @return The optional parsed JSON, or empty if the secret does not exist
+     */
+    private Optional<JsonNode> getSecretKeyValues(final ProtectedPropertyContext context) {
+        try {
+            final GetSecretValueResponse response = client.getSecretValue(builder -> builder.secretId(context.getContextName()));
+
+            if (response.secretString() == null) {
+                throw new SensitivePropertyProtectionException(String.format("Found No Secret String in AWS Secrets Manager Secret for [%s]",
+                        context.getContextKey()));
+            }
+            return Optional.of(objectMapper.readTree(response.secretString()));
+        } catch (final ResourceNotFoundException e) {
+            return Optional.empty();
+        }  catch (final SecretsManagerException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Retrieved For [%s]",

Review comment:
       ```suggestion
               throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret [%s] retrieval failed",
   ```

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.secretsmanager.model.SecretsManagerException;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class AwsSecretsManagerSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private final SecretsManagerClient client;
+    private final ObjectMapper objectMapper;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    AwsSecretsManagerSensitivePropertyProvider(final SecretsManagerClient client) {
+        super(null);
+
+        this.client = client;
+        this.objectMapper = new ObjectMapper();
+    }
+
+    @Override
+    public boolean isSupported() {
+        return client != null;
+    }
+
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+        Objects.requireNonNull(unprotectedValue, "Property value must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+
+        try {
+            writeLock.lock();
+            final String secretName = context.getContextName();
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            final ObjectNode secretObject = (ObjectNode) secretKeyValuesOptional.orElse(objectMapper.createObjectNode());

Review comment:
       Instead of returning `Optional<JsonNode>` and casting to `ObjectNode`, what do you think about having `getSecretKeyValues()` return `Optional<ObjectNode>` and handling checking and casting in that method?

##########
File path: nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AwsSecretsManagerSensitivePropertyProvider.java
##########
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.properties;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import software.amazon.awssdk.services.secretsmanager.SecretsManagerClient;
+import software.amazon.awssdk.services.secretsmanager.model.GetSecretValueResponse;
+import software.amazon.awssdk.services.secretsmanager.model.ResourceNotFoundException;
+import software.amazon.awssdk.services.secretsmanager.model.SecretsManagerException;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class AwsSecretsManagerSensitivePropertyProvider extends AbstractSensitivePropertyProvider {
+    private final SecretsManagerClient client;
+    private final ObjectMapper objectMapper;
+
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    AwsSecretsManagerSensitivePropertyProvider(final SecretsManagerClient client) {
+        super(null);
+
+        this.client = client;
+        this.objectMapper = new ObjectMapper();
+    }
+
+    @Override
+    public boolean isSupported() {
+        return client != null;
+    }
+
+    @Override
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+        Objects.requireNonNull(unprotectedValue, "Property value must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+
+        try {
+            writeLock.lock();
+            final String secretName = context.getContextName();
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            final ObjectNode secretObject = (ObjectNode) secretKeyValuesOptional.orElse(objectMapper.createObjectNode());
+
+            secretObject.put(context.getPropertyName(), unprotectedValue);
+            final String secretString = objectMapper.writeValueAsString(secretObject);
+
+            if (secretKeyValuesOptional.isPresent()) {
+                client.putSecretValue(builder -> builder.secretId(secretName).secretString(secretString));
+            } else {
+                client.createSecret(builder -> builder.name(secretName).secretString(secretString));
+            }
+            return context.getContextKey();
+        } catch (final SecretsManagerException | JsonProcessingException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Stored for [%s]", context), e);
+        } finally {
+            writeLock.unlock();
+        }
+    }
+
+    @Override
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context)
+            throws SensitivePropertyProtectionException {
+        Objects.requireNonNull(context, "Property context must be provided");
+
+        if (client == null) {
+            throw new SensitivePropertyProtectionException("AWS Secrets Manager Provider Not Configured");
+        }
+        try {
+            readLock.lock();
+
+            String propertyValue = null;
+            final Optional<JsonNode> secretKeyValuesOptional = getSecretKeyValues(context);
+            if (secretKeyValuesOptional.isPresent()) {
+                final JsonNode secretKeyValues = secretKeyValuesOptional.get();
+                final String propertyName = context.getPropertyName();
+                if (secretKeyValues.has(propertyName)) {
+                    propertyValue = secretKeyValues.get(propertyName).textValue();
+                }
+            }
+            if (propertyValue == null) {
+                throw new SensitivePropertyProtectionException(String.format("Property [%s] Not Found in AWS Secrets Manager Secret", context.getContextKey()));
+            }
+
+            return propertyValue;
+        } finally {
+            readLock.unlock();
+        }
+    }
+
+    /**
+     * Returns the optional parsed JSON from the matching secret, or empty if the secret does not exist.
+     * @param context The property context
+     * @return The optional parsed JSON, or empty if the secret does not exist
+     */
+    private Optional<JsonNode> getSecretKeyValues(final ProtectedPropertyContext context) {
+        try {
+            final GetSecretValueResponse response = client.getSecretValue(builder -> builder.secretId(context.getContextName()));
+
+            if (response.secretString() == null) {
+                throw new SensitivePropertyProtectionException(String.format("Found No Secret String in AWS Secrets Manager Secret for [%s]",
+                        context.getContextKey()));
+            }
+            return Optional.of(objectMapper.readTree(response.secretString()));
+        } catch (final ResourceNotFoundException e) {
+            return Optional.empty();
+        }  catch (final SecretsManagerException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Retrieved For [%s]",
+                    context.getContextKey()), e);
+        } catch (final JsonProcessingException e) {
+            throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret Could Not Be Parsed from [%s]",

Review comment:
       ```suggestion
               throw new SensitivePropertyProtectionException(String.format("AWS Secrets Manager Secret [%s] JSON parsing failed",
   ```




-- 
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 closed pull request #5410: NIFI-9221: Add AWS SecretsManager Sensitive Props Provider

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


   


-- 
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 pull request #5410: NIFI-9221: Add AWS SecretsManager Sensitive Props Provider

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


   Thanks for the quick response @gresockj. PR #5435 is now merged, and it looks like there are a couple conflicts with the documentation files. It looks like your most recent commit addresses the issue with the KMS Key Identifier property, so I will give this another try once you have resolved the conflicts.


-- 
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 #5410: NIFI-9221: Add AWS SecretsManager Sensitive Props Provider

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


   > @gresockj When attempting to verify this with a configured `bootstrap-aws.conf` and a protected property configured in `nifi.properties`, NiFi reported the following exception at startup:
   > 
   > ```
   > java.lang.IllegalArgumentException: There was an issue decrypting protected properties
   > 	at org.apache.nifi.NiFi.initializeProperties(NiFi.java:374)
   > 	at org.apache.nifi.NiFi.convertArgumentsToValidatedNiFiProperties(NiFi.java:342)
   > 	at org.apache.nifi.NiFi.convertArgumentsToValidatedNiFiProperties(NiFi.java:338)
   > 	at org.apache.nifi.NiFi.main(NiFi.java:330)
   > Caused by: org.apache.nifi.properties.SensitivePropertyProtectionException: AWS KMS Key [] Validation Failed
   > 	at org.apache.nifi.properties.AwsKmsSensitivePropertyProvider.validate(AwsKmsSensitivePropertyProvider.java:77)
   > 	at org.apache.nifi.properties.AwsKmsSensitivePropertyProvider.validate(AwsKmsSensitivePropertyProvider.java:34)
   > 	at org.apache.nifi.properties.ClientBasedEncodedSensitivePropertyProvider.<init>(ClientBasedEncodedSensitivePropertyProvider.java:42)
   > 	at org.apache.nifi.properties.AwsKmsSensitivePropertyProvider.<init>(AwsKmsSensitivePropertyProvider.java:38)
   > 	at org.apache.nifi.properties.StandardSensitivePropertyProviderFactory.lambda$getProvider$7(StandardSensitivePropertyProviderFactory.java:138)
   > 	at java.util.HashMap.computeIfAbsent(HashMap.java:1127)
   > 	at org.apache.nifi.properties.StandardSensitivePropertyProviderFactory.getProvider(StandardSensitivePropertyProviderFactory.java:134)
   > 	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
   > 	at java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948)
   > 	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:482)
   > 	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:472)
   > 	at java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
   > 	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
   > 	at java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:566)
   > 	at org.apache.nifi.properties.StandardSensitivePropertyProviderFactory.getSupportedSensitivePropertyProviders(StandardSensitivePropertyProviderFactory.java:175)
   > 	at org.apache.nifi.properties.NiFiPropertiesLoader.load(NiFiPropertiesLoader.java:173)
   > ```
   > 
   > The `bootstrap-aws.conf` does not have a value configured for `aws.kms.key.id` since this is using AWS Secrets Manager.
   > 
   > The problem appears to be that the `AwsKmsClientProvider` instantiates a `KmsClient`, but validation fails in `AwsKmsSensitivePropertyProvider` because the KMS Key Identifier is not configured.
   > 
   > The Azure Key Vault Key and Azure Key Vault Secrets SPP implementations have a similar challenge given that they share the same bootstrap configuration. PR #5435 adding Azure Key Vault Secret SPP addresses this issue with a method named `isMissingProperties()` in the parent `AzureClientProvider`. Implementing classes provided required property names, which determine whether or not the Client will be initialized. This in turns drives how the SPP responds to `isSupported()`. Taking a similar approach here in `AbstractAwsClientProvider` should resolve the problem.
   
   Ah, good catch.  I believe I forgot to remove the `aws.kms.key.id` from my existing configuration during testing.
   
   The implementation ended up being so similar to #5435 that I believe whichever of these two PRs land second could possibly refactor out some of the common logic between these two client providers.


-- 
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