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/10/06 16:44:05 UTC

[GitHub] [nifi] exceptionfactory commented on a change in pull request #5410: NIFI-9221: Add AWS SecretsManager Sensitive Props Provider

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