You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ex...@apache.org on 2021/07/27 20:38:28 UTC

[nifi] branch main updated: NIFI-8695: Adding context to sensitive property providers

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 2daac57  NIFI-8695: Adding context to sensitive property providers
2daac57 is described below

commit 2daac5714a6050dd872522ff74d7fe71f700db6e
Author: Joe Gresock <jg...@gmail.com>
AuthorDate: Fri Jul 9 07:14:51 2021 -0400

    NIFI-8695: Adding context to sensitive property providers
    
    This closes #5206
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../nifi/properties/BootstrapProperties.java       | 14 ++++-
 .../nifi/properties/ProtectedPropertyContext.java  | 67 +++++++++++++++++++++
 .../properties/AESSensitivePropertyProvider.java   |  6 +-
 .../properties/AWSSensitivePropertyProvider.java   |  6 +-
 .../properties/ApplicationPropertiesProtector.java |  2 +-
 ...iCorpVaultTransitSensitivePropertyProvider.java |  6 +-
 .../nifi/properties/SensitivePropertyProvider.java |  6 +-
 .../SensitivePropertyProviderFactory.java          | 11 ++++
 .../SensitivePropertyProviderFactoryAware.java     |  7 +++
 .../StandardSensitivePropertyProviderFactory.java  | 28 +++++++++
 .../AESSensitivePropertyProviderTest.groovy        | 36 +++++------
 .../properties/AWSSensitivePropertyProviderIT.java |  4 +-
 ...andardSensitivePropertyProviderFactoryTest.java | 51 +++++++++++++---
 .../nifi/authorization/AuthorizerFactoryBean.java  |  8 +--
 .../authorization/AuthorizerFactoryBeanTest.groovy |  2 +-
 .../ProtectedNiFiPropertiesGroovyTest.groovy       |  2 +-
 .../groovy/org/apache/nifi/NiFiGroovyTest.groovy   |  3 +-
 .../spring/LoginIdentityProviderFactoryBean.java   | 10 +---
 .../LoginIdentityProviderFactoryBeanTest.groovy    |  2 +-
 .../x509/X509AuthenticationProviderTest.java       |  2 +
 .../authentication/IdentityProviderFactory.java    | 31 +++++-----
 .../security/authorization/AuthorizerFactory.java  | 31 +++++-----
 ...itivePropertyProviderFactoryConfiguration.java} | 14 ++---
 ...rotectedNiFiRegistryPropertiesGroovyTest.groovy |  3 +-
 .../apache/nifi/registry/web/api/SecureLdapIT.java | 10 ++--
 .../nifi/properties/ConfigEncryptionTool.groovy    | 48 ++++++++++-----
 .../nifi/toolkit/encryptconfig/DecryptMode.groovy  |  8 ++-
 .../toolkit/encryptconfig/NiFiRegistryMode.groovy  | 15 ++---
 .../util/NiFiPropertiesEncryptor.groovy            |  5 +-
 .../NiFiRegistryAuthorizersXmlEncryptor.groovy     |  9 ++-
 ...iFiRegistryIdentityProvidersXmlEncryptor.groovy |  9 ++-
 .../util/NiFiRegistryPropertiesEncryptor.groovy    |  8 +--
 .../encryptconfig/util/PropertiesEncryptor.groovy  |  5 +-
 .../toolkit/encryptconfig/util/XmlEncryptor.groovy | 24 +++++---
 .../properties/ConfigEncryptionToolTest.groovy     | 70 ++++++++++++++--------
 .../encryptconfig/EncryptConfigMainTest.groovy     |  5 +-
 .../NiFiRegistryDecryptModeSpec.groovy             |  6 +-
 .../encryptconfig/NiFiRegistryModeSpec.groovy      | 22 +++----
 .../nifi/toolkit/encryptconfig/TestUtil.groovy     | 19 +++++-
 39 files changed, 417 insertions(+), 198 deletions(-)

diff --git a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
index 4fc9f1c..830ba05 100644
--- a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
+++ b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/BootstrapProperties.java
@@ -18,6 +18,7 @@ package org.apache.nifi.properties;
 
 import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.util.Collections;
 import java.util.Enumeration;
 import java.util.Objects;
 import java.util.Optional;
@@ -33,13 +34,22 @@ public class BootstrapProperties extends StandardReadableProperties {
     public enum BootstrapPropertyKey {
         SENSITIVE_KEY("bootstrap.sensitive.key"),
         HASHICORP_VAULT_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.hashicorp.vault.conf"),
-        AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.aws.kms.conf");
+        AWS_KMS_SENSITIVE_PROPERTY_PROVIDER_CONF("bootstrap.protection.aws.kms.conf"),
+        CONTEXT_MAPPING_PREFIX("bootstrap.protection.context.mapping.");
 
         private final String key;
 
         BootstrapPropertyKey(final String key) {
             this.key = key;
         }
+
+        /**
+         * Returns the property key.
+         * @return The property key
+         */
+        public String getKey() {
+            return key;
+        }
     }
 
     private final String propertyPrefix;
@@ -127,7 +137,7 @@ public class BootstrapProperties extends StandardReadableProperties {
     public static final BootstrapProperties EMPTY = new BootstrapProperties("", new Properties(), Paths.get("conf/bootstrap.conf")) {
         @Override
         public Set<String> getPropertyKeys() {
-            return null;
+            return Collections.EMPTY_SET;
         }
 
         @Override
diff --git a/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedPropertyContext.java b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedPropertyContext.java
new file mode 100644
index 0000000..ca2da37
--- /dev/null
+++ b/nifi-commons/nifi-property-utils/src/main/java/org/apache/nifi/properties/ProtectedPropertyContext.java
@@ -0,0 +1,67 @@
+/*
+ * 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 java.util.Objects;
+
+/**
+ * A context for protected properties, encapsulating the context name and property name.
+ */
+public class ProtectedPropertyContext {
+    private static final String DEFAULT_CONTEXT = "default";
+
+    private final String propertyName;
+    private final String contextName;
+
+    /**
+     * Creates a ProtectedPropertyContext for the given property name, with a specific context name, acting as
+     * a namespace for the property.
+     * @param propertyName The property name in this location
+     * @param contextName A custom context name.  If null, the default context will be assigned.
+     * @return A property context representing a property within a specific context
+     */
+    public static ProtectedPropertyContext contextFor(final String propertyName, final String contextName) {
+        return new ProtectedPropertyContext(propertyName, contextName);
+    }
+
+    /**
+     * Creates a ProtectedPropertyContext for the given property name, using the default context.
+     * @param propertyName The property name in this location
+     * @return A property context representing a property with the given name in the default context
+     */
+    public static ProtectedPropertyContext defaultContext(final String propertyName) {
+        return new ProtectedPropertyContext(propertyName, DEFAULT_CONTEXT);
+    }
+
+    /**
+     * Creates a property context with a property name and custom location.
+     * @param propertyName The property name
+     * @param contextName The context name.  If null, the default context will be assigned.
+     */
+    private ProtectedPropertyContext(final String propertyName, final String contextName) {
+        this.propertyName = Objects.requireNonNull(propertyName);
+        this.contextName = contextName == null ? DEFAULT_CONTEXT : contextName;
+    }
+
+    /**
+     * Returns the context key, in the format [contextName]/[propertyName]
+     * @return The context key
+     */
+    public String getContextKey() {
+        return String.format("%s/%s", contextName, propertyName);
+    }
+}
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java
index 1fd8087..543b166 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AESSensitivePropertyProvider.java
@@ -154,11 +154,12 @@ public class AESSensitivePropertyProvider extends AbstractSensitivePropertyProvi
      * Returns the encrypted cipher text.
      *
      * @param unprotectedValue the sensitive value
+     * @param context The property context, unused in this provider
      * @return the value to persist in the {@code nifi.properties} file
      * @throws SensitivePropertyProtectionException if there is an exception encrypting the value
      */
     @Override
-    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
         if (StringUtils.isBlank(unprotectedValue)) {
             throw new IllegalArgumentException("Cannot encrypt an empty value");
         }
@@ -204,11 +205,12 @@ public class AESSensitivePropertyProvider extends AbstractSensitivePropertyProvi
      * Returns the decrypted plaintext.
      *
      * @param protectedValue the cipher text read from the {@code nifi.properties} file
+     * @param context The property context, unused in this provider
      * @return the raw value to be used by the application
      * @throws SensitivePropertyProtectionException if there is an error decrypting the cipher text
      */
     @Override
-    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
         if (protectedValue == null || protectedValue.trim().length() < MIN_CIPHER_TEXT_LENGTH) {
             throw new IllegalArgumentException("Cannot decrypt a cipher text shorter than " + MIN_CIPHER_TEXT_LENGTH + " chars");
         }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
index 5075d5a..860d5c9 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/AWSSensitivePropertyProvider.java
@@ -280,10 +280,11 @@ public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvi
      * 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.
+     * @param context The context of the value (ignored in this implementation)
      * @return the value to persist in the {@code nifi.properties} file.
      */
     @Override
-    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
         if (StringUtils.isBlank(unprotectedValue)) {
             throw new IllegalArgumentException("Cannot encrypt a blank value");
         }
@@ -304,10 +305,11 @@ public class AWSSensitivePropertyProvider extends AbstractSensitivePropertyProvi
      * 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.
+     * @param context The context of the value (ignored in this implementation)
      * @return the raw value to be used by the application.
      */
     @Override
-    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
         if (StringUtils.isBlank(protectedValue)) {
             throw new IllegalArgumentException("Cannot decrypt a blank value");
         }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
index 3f03a9f..1064329 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/ApplicationPropertiesProtector.java
@@ -327,7 +327,7 @@ public class ApplicationPropertiesProtector<T extends ProtectedProperties<U>, U
 
             try {
                 final SensitivePropertyProvider sensitivePropertyProvider = getSensitivePropertyProvider(protectionScheme);
-                return sensitivePropertyProvider.unprotect(retrievedValue);
+                return sensitivePropertyProvider.unprotect(retrievedValue, ProtectedPropertyContext.defaultContext(key));
             } catch (SensitivePropertyProtectionException e) {
                 logger.error("Error unprotecting value for " + key, e);
                 throw e;
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/HashiCorpVaultTransitSensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/HashiCorpVaultTransitSensitivePropertyProvider.java
index b670cb1..6c7efd2 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/HashiCorpVaultTransitSensitivePropertyProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/HashiCorpVaultTransitSensitivePropertyProvider.java
@@ -62,11 +62,12 @@ public class HashiCorpVaultTransitSensitivePropertyProvider extends AbstractHash
      * Returns the encrypted cipher text.
      *
      * @param unprotectedValue the sensitive value
+     * @param context The property context, unused in this provider
      * @return the value to persist in the {@code nifi.properties} file
      * @throws SensitivePropertyProtectionException if there is an exception encrypting the value
      */
     @Override
-    public String protect(final String unprotectedValue) throws SensitivePropertyProtectionException {
+    public String protect(final String unprotectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
         if (StringUtils.isBlank(unprotectedValue)) {
             throw new IllegalArgumentException("Cannot encrypt an empty value");
         }
@@ -78,11 +79,12 @@ public class HashiCorpVaultTransitSensitivePropertyProvider extends AbstractHash
      * Returns the decrypted plaintext.
      *
      * @param protectedValue the cipher text read from the {@code nifi.properties} file
+     * @param context The property context, unused in this provider
      * @return the raw value to be used by the application
      * @throws SensitivePropertyProtectionException if there is an error decrypting the cipher text
      */
     @Override
-    public String unprotect(final String protectedValue) throws SensitivePropertyProtectionException {
+    public String unprotect(final String protectedValue, final ProtectedPropertyContext context) throws SensitivePropertyProtectionException {
         if (StringUtils.isBlank(protectedValue)) {
             throw new IllegalArgumentException("Cannot decrypt an empty value");
         }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java
index ef66aec..e4d7c8c 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProvider.java
@@ -44,18 +44,20 @@ public interface SensitivePropertyProvider {
      * 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
+     * @param context The context of the value
      * @return the value to persist in the {@code nifi.properties} file
      */
-    String protect(String unprotectedValue) throws SensitivePropertyProtectionException;
+    String protect(String unprotectedValue, ProtectedPropertyContext context) throws SensitivePropertyProtectionException;
 
     /**
      * 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
+     * @param context The context of the value
      * @return the raw value to be used by the application
      */
-    String unprotect(String protectedValue) throws SensitivePropertyProtectionException;
+    String unprotect(String protectedValue, ProtectedPropertyContext context) throws SensitivePropertyProtectionException;
 
     /**
      * Cleans up resources that may have been allocated/used by an SPP implementation
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactory.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactory.java
index 834e708..70ba62f 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactory.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactory.java
@@ -33,4 +33,15 @@ public interface SensitivePropertyProviderFactory {
      */
     Collection<SensitivePropertyProvider> getSupportedSensitivePropertyProviders();
 
+    /**
+     * Returns a ProtectedPropertyContext with the given property name.  The ProtectedPropertyContext's
+     * contextName will be the name found in a matching context mapping from bootstrap.conf, or 'default' if
+     * no matching mapping was found.
+     * @param groupIdentifier The identifier of a group that contains the configuration property.  The definition
+     *                        of a group depends on the type of configuration file.
+     * @param propertyName A property name
+     * @return The property context, using any mappings configured in bootstrap.conf to match against the
+     * provided group identifier (or the default context if none match).
+     */
+    ProtectedPropertyContext getPropertyContext(String groupIdentifier, String propertyName);
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactoryAware.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactoryAware.java
index 88bf1d6..c2045d7 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactoryAware.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/SensitivePropertyProviderFactoryAware.java
@@ -32,6 +32,13 @@ public class SensitivePropertyProviderFactoryAware {
         return sensitivePropertyProviderFactory;
     }
 
+    protected String decryptValue(final String cipherText, final String protectionScheme, final String propertyName, final String groupIdentifier) throws SensitivePropertyProtectionException {
+        final SensitivePropertyProviderFactory sensitivePropertyProviderFactory = getSensitivePropertyProviderFactory();
+        final ProtectedPropertyContext protectedPropertyContext = sensitivePropertyProviderFactory.getPropertyContext(groupIdentifier, propertyName);
+        return sensitivePropertyProviderFactory.getProvider(PropertyProtectionScheme.fromIdentifier(protectionScheme))
+                .unprotect(cipherText, protectedPropertyContext);
+    }
+
     /**
      * Configures and sets the SensitivePropertyProviderFactory.
      * @param keyHex An key in hex format, which some providers may use for encryption
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
index b1a52dc..cfbb90d 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/main/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactory.java
@@ -18,6 +18,7 @@ package org.apache.nifi.properties;
 
 import org.apache.nifi.properties.BootstrapProperties.BootstrapPropertyKey;
 import org.apache.nifi.util.NiFiBootstrapUtils;
+import org.apache.nifi.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -29,6 +30,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.function.Supplier;
+import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
 public class StandardSensitivePropertyProviderFactory implements SensitivePropertyProviderFactory {
@@ -37,6 +39,7 @@ public class StandardSensitivePropertyProviderFactory implements SensitiveProper
     private final Optional<String> keyHex;
     private final Supplier<BootstrapProperties> bootstrapPropertiesSupplier;
     private final Map<PropertyProtectionScheme, SensitivePropertyProvider> providerMap;
+    private Map<String, Pattern> customPropertyContextMap;
 
     /**
      * Creates a StandardSensitivePropertyProviderFactory using the default bootstrap.conf location and
@@ -74,6 +77,18 @@ public class StandardSensitivePropertyProviderFactory implements SensitiveProper
         this.keyHex = Optional.ofNullable(keyHex);
         this.bootstrapPropertiesSupplier = bootstrapPropertiesSupplier == null ? () -> null : bootstrapPropertiesSupplier;
         this.providerMap = new HashMap<>();
+        this.customPropertyContextMap = null;
+    }
+
+    private void populateCustomPropertyContextMap() {
+        final BootstrapProperties bootstrapProperties = getBootstrapProperties();
+        customPropertyContextMap = new HashMap<>();
+        final String contextMappingKeyPrefix = BootstrapPropertyKey.CONTEXT_MAPPING_PREFIX.getKey();
+        bootstrapProperties.getPropertyKeys().stream()
+                .filter(k -> k.contains(contextMappingKeyPrefix))
+                .forEach(k -> {
+                    customPropertyContextMap.put(StringUtils.substringAfter(k, contextMappingKeyPrefix), Pattern.compile(bootstrapProperties.getProperty(k)));
+                });
     }
 
     private String getKeyHex() {
@@ -122,4 +137,17 @@ public class StandardSensitivePropertyProviderFactory implements SensitiveProper
                 .collect(Collectors.toList());
     }
 
+    @Override
+    public ProtectedPropertyContext getPropertyContext(final String groupIdentifier, final String propertyName) {
+        if (customPropertyContextMap == null) {
+            populateCustomPropertyContextMap();
+        }
+        final String contextName = customPropertyContextMap.entrySet().stream()
+                .filter(entry -> entry.getValue().matcher(groupIdentifier).find())
+                .map(Map.Entry::getKey)
+                .findFirst()
+                .orElse(null);
+        return ProtectedPropertyContext.contextFor(propertyName, contextName);
+    }
+
 }
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy b/nifi-commons/nifi-sensitive-property-provider/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy
index 7bc22b6..cef68b1 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy
@@ -47,6 +47,8 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
 
     private static final SecureRandom secureRandom = new SecureRandom()
 
+    private static final ProtectedPropertyContext PROPERTY_CONTEXT = ProtectedPropertyContext.defaultContext("propertyName")
+
     private static final Base64.Encoder encoder = Base64.encoder
     private static final Base64.Decoder decoder = Base64.decoder
 
@@ -139,7 +141,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         Map<Integer, String> CIPHER_TEXTS = KEY_SIZES.collectEntries { int keySize ->
             SensitivePropertyProvider spp = new AESSensitivePropertyProvider(Hex.decode(getKeyOfSize(keySize)))
             logger.info("Initialized ${spp.name} with key size ${keySize}")
-            [(keySize): spp.protect(PLAINTEXT)]
+            [(keySize): spp.protect(PLAINTEXT, PROPERTY_CONTEXT)]
         }
         CIPHER_TEXTS.each { ks, ct -> logger.info("Encrypted for ${ks} length key: ${ct}") }
 
@@ -171,7 +173,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
             logger.info("Initialized ${spp.name} with key size ${keySize}")
             EMPTY_PLAINTEXTS.each { String emptyPlaintext ->
                 def msg = shouldFail(IllegalArgumentException) {
-                    spp.protect(emptyPlaintext)
+                    spp.protect(emptyPlaintext, PROPERTY_CONTEXT)
                 }
                 logger.expected("${msg} for keySize ${keySize} and plaintext [${emptyPlaintext}]")
 
@@ -203,7 +205,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         Map<Integer, String> plaintexts = CIPHER_TEXTS.collectEntries { int keySize, String cipherText ->
             SensitivePropertyProvider spp = new AESSensitivePropertyProvider(Hex.decode(getKeyOfSize(keySize)))
             logger.info("Initialized ${spp.name} with key size ${keySize}")
-            [(keySize): spp.unprotect(cipherText)]
+            [(keySize): spp.unprotect(cipherText, PROPERTY_CONTEXT)]
         }
         plaintexts.each { ks, pt -> logger.info("Decrypted for ${ks} length key: ${pt}") }
 
@@ -227,7 +229,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
             logger.info("Initialized ${spp.name} with key size ${keySize}")
             EMPTY_CIPHER_TEXTS.each { String emptyCipherText ->
                 def msg = shouldFail(IllegalArgumentException) {
-                    spp.unprotect(emptyCipherText)
+                    spp.unprotect(emptyCipherText, PROPERTY_CONTEXT)
                 }
                 logger.expected("${msg} for keySize ${keySize} and cipher text [${emptyCipherText}]")
 
@@ -259,7 +261,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         Map<Integer, String> plaintexts = CIPHER_TEXTS.collectEntries { int keySize, String cipherText ->
             SensitivePropertyProvider spp = new AESSensitivePropertyProvider(Hex.decode(getKeyOfSize(keySize)))
             logger.info("Initialized ${spp.name} with key size ${keySize}")
-            [(keySize): spp.unprotect("\t" + cipherText + "\n")]
+            [(keySize): spp.unprotect("\t" + cipherText + "\n", PROPERTY_CONTEXT)]
         }
         plaintexts.each { ks, pt -> logger.info("Decrypted for ${ks} length key: ${pt}") }
 
@@ -276,13 +278,13 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         KEY_SIZES.each { int keySize ->
             SensitivePropertyProvider spp = new AESSensitivePropertyProvider(Hex.decode(getKeyOfSize(keySize)))
             logger.info("Initialized ${spp.name} with key size ${keySize}")
-            String cipherText = spp.protect(PLAINTEXT)
+            String cipherText = spp.protect(PLAINTEXT, PROPERTY_CONTEXT)
             // Swap two characters in the cipher text
             final String MALFORMED_CIPHER_TEXT = manipulateString(cipherText, 25, 28)
             logger.info("Manipulated ${cipherText} to\n${MALFORMED_CIPHER_TEXT.padLeft(163)}")
 
             def msg = shouldFail(SensitivePropertyProtectionException) {
-                spp.unprotect(MALFORMED_CIPHER_TEXT)
+                spp.unprotect(MALFORMED_CIPHER_TEXT, PROPERTY_CONTEXT)
             }
             logger.expected("${msg} for keySize ${keySize} and cipher text [${MALFORMED_CIPHER_TEXT}]")
 
@@ -300,14 +302,14 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         KEY_SIZES.each { int keySize ->
             SensitivePropertyProvider spp = new AESSensitivePropertyProvider(Hex.decode(getKeyOfSize(keySize)))
             logger.info("Initialized ${spp.name} with key size ${keySize}")
-            String cipherText = spp.protect(PLAINTEXT)
+            String cipherText = spp.protect(PLAINTEXT, PROPERTY_CONTEXT)
 
             // Remove the IV from the "complete" cipher text
             final String MISSING_IV_CIPHER_TEXT = cipherText[18..-1]
             logger.info("Manipulated ${cipherText} to\n${MISSING_IV_CIPHER_TEXT.padLeft(172)}")
 
             def msg = shouldFail(IllegalArgumentException) {
-                spp.unprotect(MISSING_IV_CIPHER_TEXT)
+                spp.unprotect(MISSING_IV_CIPHER_TEXT, PROPERTY_CONTEXT)
             }
             logger.expected("${msg} for keySize ${keySize} and cipher text [${MISSING_IV_CIPHER_TEXT}]")
 
@@ -316,7 +318,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
             logger.info("Manipulated ${cipherText} to\n${MISSING_IV_CIPHER_TEXT_WITH_DELIMITER.padLeft(172)}")
 
             def msgWithDelimiter = shouldFail(IllegalArgumentException) {
-                spp.unprotect(MISSING_IV_CIPHER_TEXT_WITH_DELIMITER)
+                spp.unprotect(MISSING_IV_CIPHER_TEXT_WITH_DELIMITER, PROPERTY_CONTEXT)
             }
             logger.expected("${msgWithDelimiter} for keySize ${keySize} and cipher text [${MISSING_IV_CIPHER_TEXT_WITH_DELIMITER}]")
 
@@ -347,7 +349,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
             logger.info("Initialized ${spp.name} with key size ${keySize}")
             EMPTY_CIPHER_TEXTS.each { String emptyCipherText ->
                 def msg = shouldFail(IllegalArgumentException) {
-                    spp.unprotect(emptyCipherText)
+                    spp.unprotect(emptyCipherText, PROPERTY_CONTEXT)
                 }
                 logger.expected("${msg} for keySize ${keySize} and cipher text [${emptyCipherText}]")
 
@@ -366,13 +368,13 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         KEY_SIZES.each { int keySize ->
             SensitivePropertyProvider spp = new AESSensitivePropertyProvider(Hex.decode(getKeyOfSize(keySize)))
             logger.info("Initialized ${spp.name} with key size ${keySize}")
-            String cipherText = spp.protect(PLAINTEXT)
+            String cipherText = spp.protect(PLAINTEXT, PROPERTY_CONTEXT)
             // Swap two characters in the IV
             final String MALFORMED_IV_CIPHER_TEXT = manipulateString(cipherText, 8, 11)
             logger.info("Manipulated ${cipherText} to\n${MALFORMED_IV_CIPHER_TEXT.padLeft(163)}")
 
             def msg = shouldFail(SensitivePropertyProtectionException) {
-                spp.unprotect(MALFORMED_IV_CIPHER_TEXT)
+                spp.unprotect(MALFORMED_IV_CIPHER_TEXT, PROPERTY_CONTEXT)
             }
             logger.expected("${msg} for keySize ${keySize} and cipher text [${MALFORMED_IV_CIPHER_TEXT}]")
 
@@ -456,7 +458,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
 
         // Act
         def encryptedValues = values.collect { String v ->
-            def encryptedValue = spp.protect(v)
+            def encryptedValue = spp.protect(v, PROPERTY_CONTEXT)
             logger.info("${v} -> ${encryptedValue}")
             def (String iv, String cipherText) = encryptedValue.tokenize("||")
             logger.info("Normal Base64 encoding would be ${encoder.encodeToString(decoder.decode(iv))}||${encoder.encodeToString(decoder.decode(cipherText))}")
@@ -464,7 +466,7 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         }
 
         // Assert
-        assert values == encryptedValues.collect { spp.unprotect(it) }
+        assert values == encryptedValues.collect { spp.unprotect(it, PROPERTY_CONTEXT) }
     }
 
     /**
@@ -484,9 +486,9 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase {
         SensitivePropertyProvider spp = new AESSensitivePropertyProvider(key)
 
         // Act
-        String rawValue = spp.unprotect(cipherText)
+        String rawValue = spp.unprotect(cipherText, PROPERTY_CONTEXT)
         logger.info("Decrypted ${cipherText} to ${rawValue}")
-        String rawUnpaddedValue = spp.unprotect(unpaddedCipherText)
+        String rawUnpaddedValue = spp.unprotect(unpaddedCipherText, PROPERTY_CONTEXT)
         logger.info("Decrypted ${unpaddedCipherText} to ${rawUnpaddedValue}")
 
         // Assert
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
index 31125e3..8ba8bfb 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/AWSSensitivePropertyProviderIT.java
@@ -121,9 +121,9 @@ public class AWSSensitivePropertyProviderIT {
 
     private static void runEncryptDecryptTest() {
         logger.info("Plaintext: " + SAMPLE_PLAINTEXT);
-        String protectedValue = spp.protect(SAMPLE_PLAINTEXT);
+        String protectedValue = spp.protect(SAMPLE_PLAINTEXT, ProtectedPropertyContext.defaultContext("property"));
         logger.info("Protected Value: " + protectedValue);
-        String unprotectedValue = spp.unprotect(protectedValue);
+        String unprotectedValue = spp.unprotect(protectedValue, ProtectedPropertyContext.defaultContext("property"));
         logger.info("Unprotected Value: " + unprotectedValue);
 
         Assert.assertEquals(SAMPLE_PLAINTEXT, unprotectedValue);
diff --git a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactoryTest.java b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactoryTest.java
index 81995bf..0f012b1 100644
--- a/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactoryTest.java
+++ b/nifi-commons/nifi-sensitive-property-provider/src/test/java/org/apache/nifi/properties/StandardSensitivePropertyProviderFactoryTest.java
@@ -30,6 +30,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
 import java.security.Security;
 import java.util.Properties;
 import java.util.function.Supplier;
@@ -53,6 +54,7 @@ public class StandardSensitivePropertyProviderFactoryTest {
     private static Path bootstrapConf;
     private static Path hashicorpVaultBootstrapConf;
     private static Path nifiProperties;
+    private static String defaultBootstrapContents;
 
     private static NiFiProperties niFiProperties;
 
@@ -65,18 +67,30 @@ public class StandardSensitivePropertyProviderFactoryTest {
 
         nifiProperties = Files.createTempFile("nifi", ".properties").toAbsolutePath();
 
-        bootstrapConf = Files.move(bootstrapConf, tempConfDir.resolve("bootstrap.conf"));
         nifiProperties = Files.move(nifiProperties, tempConfDir.resolve("nifi.properties"));
 
-        final String bootstrapConfText = String.format("%s=%s\n%s=%s",
+        defaultBootstrapContents = String.format("%s=%s\n%s=%s",
                 "nifi.bootstrap.sensitive.key", BOOTSTRAP_KEY_HEX,
                 "nifi.bootstrap.protection.hashicorp.vault.conf", FilenameUtils.separatorsToUnix(hashicorpVaultBootstrapConf.toString()));
-        IOUtil.writeText(bootstrapConfText, bootstrapConf.toFile());
+        bootstrapConf = writeDefaultBootstrapConf();
         System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, FilenameUtils.separatorsToUnix(nifiProperties.toString()));
 
         niFiProperties = new NiFiProperties();
     }
 
+    private static Path writeDefaultBootstrapConf() throws IOException {
+        return writeBootstrapConf(defaultBootstrapContents);
+    }
+
+    private static Path writeBootstrapConf(final String contents) throws IOException {
+        final Path tempBootstrapConf = Files.createTempFile("bootstrap", ".conf").toAbsolutePath();
+        final Path bootstrapConf = Files.move(tempBootstrapConf, tempConfDir.resolve("bootstrap.conf"), StandardCopyOption.REPLACE_EXISTING);
+
+        final String bootstrapConfText = String.format(contents);
+        IOUtil.writeText(bootstrapConfText, bootstrapConf.toFile());
+        return bootstrapConf;
+    }
+
     @AfterClass
     public static void tearDownOnce() throws IOException {
         Files.deleteIfExists(bootstrapConf);
@@ -123,6 +137,26 @@ public class StandardSensitivePropertyProviderFactoryTest {
     }
 
     @Test
+    public void testGetPropertyContextUnconfigured() {
+        configureDefaultFactory();
+        assertEquals("default/prop", factory.getPropertyContext("ldap-provider", "prop").getContextKey());
+
+    }
+
+    @Test
+    public void testGetPropertyContext() throws IOException {
+        configureDefaultFactory();
+        writeBootstrapConf(defaultBootstrapContents + "\n" +
+                "nifi.bootstrap.protection.context.mapping.ldap=ldap-.*");
+        try {
+            assertEquals("ldap/prop", factory.getPropertyContext("ldap-provider", "prop").getContextKey());
+            assertEquals("ldap/prop", factory.getPropertyContext("ldap-user-group-provider", "prop").getContextKey());
+        } finally {
+            writeDefaultBootstrapConf();
+        }
+    }
+
+    @Test
     public void testHashicorpVaultTransit() throws IOException {
         configureDefaultFactory();
         final Properties properties = new Properties();
@@ -169,14 +203,15 @@ public class StandardSensitivePropertyProviderFactoryTest {
     @Test
     public void testAES_GCM() throws IOException {
         configureDefaultFactory();
+        final ProtectedPropertyContext context = ProtectedPropertyContext.defaultContext("propertyName");
 
         final SensitivePropertyProvider spp = factory.getProvider(PropertyProtectionScheme.AES_GCM);
         assertNotNull(spp);
         assertTrue(spp.isSupported());
 
         final String cleartext = "test";
-        assertEquals(cleartext, spp.unprotect(spp.protect(cleartext)));
-        assertNotEquals(cleartext, spp.protect(cleartext));
+        assertEquals(cleartext, spp.unprotect(spp.protect(cleartext, context), context));
+        assertNotEquals(cleartext, spp.protect(cleartext, context));
         assertEquals(AES_GCM_128, spp.getIdentifierKey());
 
         // Key is now different
@@ -186,8 +221,8 @@ public class StandardSensitivePropertyProviderFactoryTest {
         assertTrue(sppAdHocKey.isSupported());
         assertEquals(AES_GCM_128, sppAdHocKey.getIdentifierKey());
 
-        assertNotEquals(spp.protect(cleartext), sppAdHocKey.protect(cleartext));
-        assertEquals(cleartext, sppAdHocKey.unprotect(sppAdHocKey.protect(cleartext)));
+        assertNotEquals(spp.protect(cleartext, context), sppAdHocKey.protect(cleartext, context));
+        assertEquals(cleartext, sppAdHocKey.unprotect(sppAdHocKey.protect(cleartext, context), context));
 
         // This should use the same keyHex as the second one
         configureAdHocKeyAndPropertiesFactory();
@@ -196,6 +231,6 @@ public class StandardSensitivePropertyProviderFactoryTest {
         assertTrue(sppKeyProperties.isSupported());
         assertEquals(AES_GCM_128, sppKeyProperties.getIdentifierKey());
 
-        assertEquals(cleartext, sppKeyProperties.unprotect(sppKeyProperties.protect(cleartext)));
+        assertEquals(cleartext, sppKeyProperties.unprotect(sppKeyProperties.protect(cleartext, context), context));
     }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
index 0056ef9..88bb5dd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/main/java/org/apache/nifi/authorization/AuthorizerFactoryBean.java
@@ -25,8 +25,6 @@ import org.apache.nifi.authorization.generated.Authorizers;
 import org.apache.nifi.authorization.generated.Property;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.properties.PropertyProtectionScheme;
-import org.apache.nifi.properties.SensitivePropertyProtectionException;
 import org.apache.nifi.properties.SensitivePropertyProviderFactoryAware;
 import org.apache.nifi.security.xml.XmlUtils;
 import org.apache.nifi.util.NiFiProperties;
@@ -383,7 +381,7 @@ public class AuthorizerFactoryBean extends SensitivePropertyProviderFactoryAware
 
         for (final Property property : properties) {
             if (!StringUtils.isBlank(property.getEncryption())) {
-                String decryptedValue = decryptValue(property.getValue(), property.getEncryption());
+                String decryptedValue = decryptValue(property.getValue(), property.getEncryption(), property.getName(), identifier);
                 authorizerProperties.put(property.getName(), decryptedValue);
             } else {
                 authorizerProperties.put(property.getName(), property.getValue());
@@ -481,10 +479,6 @@ public class AuthorizerFactoryBean extends SensitivePropertyProviderFactoryAware
         };
     }
 
-    private String decryptValue(final String cipherText, final String protectionScheme) throws SensitivePropertyProtectionException {
-        return getSensitivePropertyProviderFactory().getProvider(PropertyProtectionScheme.fromIdentifier(protectionScheme)).unprotect(cipherText);
-    }
-
     @Override
     public Class getObjectType() {
         return Authorizer.class;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/test/groovy/org/apache/nifi/authorization/AuthorizerFactoryBeanTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/test/groovy/org/apache/nifi/authorization/AuthorizerFactoryBeanTest.groovy
index 9a548ba..85f85ef 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/test/groovy/org/apache/nifi/authorization/AuthorizerFactoryBeanTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-authorizer/src/test/groovy/org/apache/nifi/authorization/AuthorizerFactoryBeanTest.groovy
@@ -77,7 +77,7 @@ class AuthorizerFactoryBeanTest extends GroovyTestCase {
         logger.info("Cipher text: ${CIPHER_TEXT}")
 
         // Act
-        String decrypted = bean.decryptValue(CIPHER_TEXT, ENCRYPTION_SCHEME)
+        String decrypted = bean.decryptValue(CIPHER_TEXT, ENCRYPTION_SCHEME, "propertyName", "ldap-user-group-provider")
         logger.info("Decrypted ${CIPHER_TEXT} -> ${decrypted}")
 
         // Assert
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
index 6c49f6d..adb309c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/ProtectedNiFiPropertiesGroovyTest.groovy
@@ -451,7 +451,7 @@ class ProtectedNiFiPropertiesGroovyTest extends GroovyTestCase {
                 .findAll { String key, String scheme -> scheme == spp.identifierKey }
                 .keySet().collect { String key ->
             try {
-                String rawValue = spp.unprotect(properties.getProperty(key))
+                String rawValue = spp.unprotect(properties.getProperty(key), ProtectedPropertyContext.defaultContext(key))
                 return
             } catch (SensitivePropertyProtectionException e) {
                 logger.expected("Caught a malformed value for ${key}")
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/groovy/org/apache/nifi/NiFiGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/groovy/org/apache/nifi/NiFiGroovyTest.groovy
index 5dda4e1..7fdc4dd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/groovy/org/apache/nifi/NiFiGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-runtime/src/test/groovy/org/apache/nifi/NiFiGroovyTest.groovy
@@ -21,6 +21,7 @@ import ch.qos.logback.core.AppenderBase
 import org.apache.nifi.properties.ApplicationPropertiesProtector
 import org.apache.nifi.properties.NiFiPropertiesLoader
 import org.apache.nifi.properties.PropertyProtectionScheme
+import org.apache.nifi.properties.ProtectedPropertyContext
 import org.apache.nifi.properties.SensitivePropertyProvider
 import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory
 import org.apache.nifi.util.NiFiProperties
@@ -216,7 +217,7 @@ class NiFiGroovyTest extends GroovyTestCase {
                 .getProvider(PropertyProtectionScheme.AES_GCM)
         def map = encryptedProperties.getPropertyKeys().collectEntries { String key ->
             if (encryptedProperties.getProperty(key + ApplicationPropertiesProtector.PROTECTED_KEY_SUFFIX) == spp.getIdentifierKey()) {
-                [(key): spp.unprotect(encryptedProperties.getProperty(key))]
+                [(key): spp.unprotect(encryptedProperties.getProperty(key), ProtectedPropertyContext.defaultContext(key))]
             } else if (!key.endsWith(ApplicationPropertiesProtector.PROTECTED_KEY_SUFFIX)) {
                 [(key): encryptedProperties.getProperty(key)]
             }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
index 3e581c2..b156804 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBean.java
@@ -32,8 +32,6 @@ import org.apache.nifi.authentication.generated.Provider;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.properties.PropertyProtectionScheme;
-import org.apache.nifi.properties.SensitivePropertyProtectionException;
 import org.apache.nifi.properties.SensitivePropertyProviderFactoryAware;
 import org.apache.nifi.security.xml.XmlUtils;
 import org.apache.nifi.util.NiFiProperties;
@@ -209,7 +207,8 @@ public class LoginIdentityProviderFactoryBean extends SensitivePropertyProviderF
 
         for (final Property property : provider.getProperty()) {
             if (!StringUtils.isBlank(property.getEncryption())) {
-                String decryptedValue = decryptValue(property.getValue(), property.getEncryption());
+                String decryptedValue = decryptValue(property.getValue(), property.getEncryption(), property.getName(), provider
+                        .getIdentifier());
                 providerProperties.put(property.getName(), decryptedValue);
             } else {
                 providerProperties.put(property.getName(), property.getValue());
@@ -219,11 +218,6 @@ public class LoginIdentityProviderFactoryBean extends SensitivePropertyProviderF
         return new StandardLoginIdentityProviderConfigurationContext(provider.getIdentifier(), providerProperties);
     }
 
-    private String decryptValue(final String cipherText, final String protectionScheme) throws SensitivePropertyProtectionException {
-        return getSensitivePropertyProviderFactory().getProvider(PropertyProtectionScheme.fromIdentifier(protectionScheme))
-                .unprotect(cipherText);
-    }
-
     private void performMethodInjection(final LoginIdentityProvider instance, final Class loginIdentityProviderClass)
             throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBeanTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBeanTest.groovy
index eca7efb..79a55f1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBeanTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/spring/LoginIdentityProviderFactoryBeanTest.groovy
@@ -78,7 +78,7 @@ class LoginIdentityProviderFactoryBeanTest extends GroovyTestCase {
         logger.info("Cipher text: ${CIPHER_TEXT}")
 
         // Act
-        String decrypted = bean.decryptValue(CIPHER_TEXT, ENCRYPTION_SCHEME)
+        String decrypted = bean.decryptValue(CIPHER_TEXT, ENCRYPTION_SCHEME, "propertyName", "ldap-provider")
         logger.info("Decrypted ${CIPHER_TEXT} -> ${decrypted}")
 
         // Assert
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java
index 83ba270..ec622a9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java
@@ -70,6 +70,8 @@ public class X509AuthenticationProviderTest {
 
     @Before
     public void setup() {
+
+        System.clearProperty(NiFiProperties.PROPERTIES_FILE_PATH);
         extractor = new SubjectDnX509PrincipalExtractor();
 
         certificateIdentityProvider = mock(X509IdentityProvider.class);
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
index 9a06d59..6c78a32 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authentication/IdentityProviderFactory.java
@@ -17,8 +17,10 @@
 package org.apache.nifi.registry.security.authentication;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.properties.PropertyProtectionScheme;
 import org.apache.nifi.properties.SensitivePropertyProtectionException;
 import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
 import org.apache.nifi.registry.extension.ExtensionManager;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
 import org.apache.nifi.registry.security.authentication.annotation.IdentityProviderContext;
@@ -71,7 +73,7 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
 
     private NiFiRegistryProperties properties;
     private ExtensionManager extensionManager;
-    private SensitivePropertyProvider sensitivePropertyProvider;
+    private SensitivePropertyProviderFactory sensitivePropertyProviderFactory;
     private IdentityProvider identityProvider;
     private final Map<String, IdentityProvider> identityProviders = new HashMap<>();
 
@@ -79,10 +81,10 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
     public IdentityProviderFactory(
             final NiFiRegistryProperties properties,
             final ExtensionManager extensionManager,
-            @Nullable final SensitivePropertyProvider sensitivePropertyProvider) {
+            @Nullable final SensitivePropertyProviderFactory sensitivePropertyProviderFactory) {
         this.properties = properties;
         this.extensionManager = extensionManager;
-        this.sensitivePropertyProvider = sensitivePropertyProvider;
+        this.sensitivePropertyProviderFactory = sensitivePropertyProviderFactory;
 
         if (this.properties == null) {
             throw new IllegalStateException("NiFiRegistryProperties cannot be null");
@@ -194,7 +196,7 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
 
         for (final Property property : provider.getProperty()) {
             if (!StringUtils.isBlank(property.getEncryption())) {
-                String decryptedValue = decryptValue(property.getValue(), property.getEncryption());
+                String decryptedValue = decryptValue(property.getValue(), property.getEncryption(), property.getName(), provider.getIdentifier());
                 providerProperties.put(property.getName(), decryptedValue);
             } else {
                 providerProperties.put(property.getName(), property.getValue());
@@ -270,22 +272,19 @@ public class IdentityProviderFactory implements IdentityProviderLookup, Disposab
         }
     }
 
-    private String decryptValue(String cipherText, String encryptionScheme) throws SensitivePropertyProtectionException {
-        if (sensitivePropertyProvider == null) {
-            throw new SensitivePropertyProtectionException("Sensitive Property Provider dependency was never wired, so protected " +
+    private String decryptValue(final String cipherText, final String encryptionScheme, final String propertyName, final String groupIdentifier) throws SensitivePropertyProtectionException {
+        if (sensitivePropertyProviderFactory == null) {
+            throw new SensitivePropertyProtectionException("Sensitive Property Provider Factory dependency was never wired, so protected " +
                     "properties cannot be decrypted. This usually indicates that a master key for this NiFi Registry was not " +
                     "detected and configured during the bootstrap startup sequence. Contact the system administrator.");
         }
-
-        if (!sensitivePropertyProvider.getIdentifierKey().equalsIgnoreCase(encryptionScheme)) {
-            throw new SensitivePropertyProtectionException("Identity Provider configuration XML was protected using " +
-                    encryptionScheme +
-                    ", but the configured Sensitive Property Provider supports " +
-                    sensitivePropertyProvider.getIdentifierKey() +
-                    ". Cannot configure this Identity Provider due to failing to decrypt protected configuration properties.");
+        try {
+            final SensitivePropertyProvider sensitivePropertyProvider = sensitivePropertyProviderFactory.getProvider(PropertyProtectionScheme.fromIdentifier(encryptionScheme));
+            return sensitivePropertyProvider.unprotect(cipherText, sensitivePropertyProviderFactory.getPropertyContext(groupIdentifier, propertyName));
+        } catch (final IllegalArgumentException e) {
+            throw new SensitivePropertyProtectionException(String.format("Identity Provider configuration XML was protected using %s, which is not supported. " +
+                    "Cannot configure this Identity Provider due to failing to decrypt protected configuration properties.", encryptionScheme));
         }
-
-        return sensitivePropertyProvider.unprotect(cipherText);
     }
 
 }
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
index 3d1e790..5704c4a 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
@@ -18,8 +18,10 @@ package org.apache.nifi.registry.security.authorization;
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.Validate;
+import org.apache.nifi.properties.PropertyProtectionScheme;
 import org.apache.nifi.properties.SensitivePropertyProtectionException;
 import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
 import org.apache.nifi.registry.extension.ExtensionClassLoader;
 import org.apache.nifi.registry.extension.ExtensionCloseable;
 import org.apache.nifi.registry.extension.ExtensionManager;
@@ -103,7 +105,7 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
 
     private final NiFiRegistryProperties properties;
     private final ExtensionManager extensionManager;
-    private final SensitivePropertyProvider sensitivePropertyProvider;
+    private final SensitivePropertyProviderFactory sensitivePropertyProviderFactory;
     private final RegistryService registryService;
     private final DataSource dataSource;
     private final IdentityMapper identityMapper;
@@ -117,14 +119,14 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
     public AuthorizerFactory(
             final NiFiRegistryProperties properties,
             final ExtensionManager extensionManager,
-            @Nullable final SensitivePropertyProvider sensitivePropertyProvider,
+            @Nullable final SensitivePropertyProviderFactory sensitivePropertyProviderFactory,
             final RegistryService registryService,
             final DataSource dataSource,
             final IdentityMapper identityMapper) {
 
         this.properties = Validate.notNull(properties);
         this.extensionManager = Validate.notNull(extensionManager);
-        this.sensitivePropertyProvider = sensitivePropertyProvider;
+        this.sensitivePropertyProviderFactory = sensitivePropertyProviderFactory;
         this.registryService = Validate.notNull(registryService);
         this.dataSource = Validate.notNull(dataSource);
         this.identityMapper = Validate.notNull(identityMapper);
@@ -313,7 +315,7 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
 
         for (final Prop property : properties) {
             if (!StringUtils.isBlank(property.getEncryption())) {
-                String decryptedValue = decryptValue(property.getValue(), property.getEncryption());
+                String decryptedValue = decryptValue(property.getValue(), property.getEncryption(), property.getName(), identifier);
                 authorizerProperties.put(property.getName(), decryptedValue);
             } else {
                 authorizerProperties.put(property.getName(), property.getValue());
@@ -508,22 +510,19 @@ public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyP
         }
     }
 
-    private String decryptValue(String cipherText, String encryptionScheme) throws SensitivePropertyProtectionException {
-        if (sensitivePropertyProvider == null) {
-            throw new SensitivePropertyProtectionException("Sensitive Property Provider dependency was never wired, so protected" +
+    private String decryptValue(final String cipherText, final String encryptionScheme, final String propertyName, final String groupIdentifier) throws SensitivePropertyProtectionException {
+        if (sensitivePropertyProviderFactory == null) {
+            throw new SensitivePropertyProtectionException("Sensitive Property Provider Factory dependency was never wired, so protected " +
                     "properties cannot be decrypted. This usually indicates that a master key for this NiFi Registry was not " +
                     "detected and configured during the bootstrap startup sequence. Contact the system administrator.");
         }
-
-        if (!sensitivePropertyProvider.getIdentifierKey().equalsIgnoreCase(encryptionScheme)) {
-            throw new SensitivePropertyProtectionException("Identity Provider configuration XML was protected using " +
-                    encryptionScheme +
-                    ", but the configured Sensitive Property Provider supports " +
-                    sensitivePropertyProvider.getIdentifierKey() +
-                    ". Cannot configure this Identity Provider due to failing to decrypt protected configuration properties.");
+        try {
+            final SensitivePropertyProvider sensitivePropertyProvider = sensitivePropertyProviderFactory.getProvider(PropertyProtectionScheme.fromIdentifier(encryptionScheme));
+            return sensitivePropertyProvider.unprotect(cipherText, sensitivePropertyProviderFactory.getPropertyContext(groupIdentifier, propertyName));
+        } catch (final IllegalArgumentException e) {
+            throw new SensitivePropertyProtectionException(String.format("Authorizer configuration XML was protected using %s, which is not supported. " +
+                    "Cannot configure this Authorizer due to failing to decrypt protected configuration properties.", encryptionScheme));
         }
-
-        return sensitivePropertyProvider.unprotect(cipherText);
     }
 
 
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderConfiguration.java b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderFactoryConfiguration.java
similarity index 86%
rename from nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderConfiguration.java
rename to nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderFactoryConfiguration.java
index 055b71e..2e0d23e 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderConfiguration.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/crypto/SensitivePropertyProviderFactoryConfiguration.java
@@ -16,9 +16,8 @@
  */
 package org.apache.nifi.registry.security.crypto;
 
-import org.apache.nifi.properties.PropertyProtectionScheme;
 import org.apache.nifi.properties.SensitivePropertyProtectionException;
-import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
 import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
 import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapUtils;
 import org.slf4j.Logger;
@@ -30,10 +29,8 @@ import org.springframework.context.annotation.Configuration;
 import java.io.IOException;
 
 @Configuration
-public class SensitivePropertyProviderConfiguration {
-    private static final Logger logger = LoggerFactory.getLogger(SensitivePropertyProviderConfiguration.class);
-
-    private static final PropertyProtectionScheme DEFAULT_SCHEME = PropertyProtectionScheme.AES_GCM;
+public class SensitivePropertyProviderFactoryConfiguration {
+    private static final Logger logger = LoggerFactory.getLogger(SensitivePropertyProviderFactoryConfiguration.class);
 
     @Autowired(required = false)
     private CryptoKeyProvider masterKeyProvider;
@@ -43,7 +40,7 @@ public class SensitivePropertyProviderConfiguration {
      *         or null if the master key is not present.
      */
     @Bean
-    public SensitivePropertyProvider getProvider() {
+    public SensitivePropertyProviderFactory getProvider() {
         if (masterKeyProvider == null || masterKeyProvider.isEmpty()) {
             // This NiFi Registry was not configured with a master key, so the assumption is
             // the optional Spring bean normally provided by this method will never be needed
@@ -62,8 +59,7 @@ public class SensitivePropertyProviderConfiguration {
                         } catch (IOException e) {
                             throw new SensitivePropertyProtectionException("Error creating Sensitive Property Provider", e);
                         }
-                    })
-                    .getProvider(DEFAULT_SCHEME);
+                    });
         } catch (final MissingCryptoKeyException e) {
             logger.warn("Error creating Sensitive Property Provider", e);
             throw new SensitivePropertyProtectionException("Error creating Sensitive Property Provider", e);
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-properties/src/test/groovy/org/apache/nifi/registry/properties/ProtectedNiFiRegistryPropertiesGroovyTest.groovy b/nifi-registry/nifi-registry-core/nifi-registry-properties/src/test/groovy/org/apache/nifi/registry/properties/ProtectedNiFiRegistryPropertiesGroovyTest.groovy
index b1bb526..1361a27 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-properties/src/test/groovy/org/apache/nifi/registry/properties/ProtectedNiFiRegistryPropertiesGroovyTest.groovy
+++ b/nifi-registry/nifi-registry-core/nifi-registry-properties/src/test/groovy/org/apache/nifi/registry/properties/ProtectedNiFiRegistryPropertiesGroovyTest.groovy
@@ -19,6 +19,7 @@ package org.apache.nifi.registry.properties
 import org.apache.nifi.properties.ApplicationPropertiesProtector
 import org.apache.nifi.properties.MultipleSensitivePropertyProtectionException
 import org.apache.nifi.properties.PropertyProtectionScheme
+import org.apache.nifi.properties.ProtectedPropertyContext
 import org.apache.nifi.properties.SensitivePropertyProtectionException
 import org.apache.nifi.properties.SensitivePropertyProvider
 import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory
@@ -356,7 +357,7 @@ class ProtectedNiFiRegistryPropertiesGroovyTest extends GroovyTestCase {
                 .keySet()
                 .findAll { String key ->
             try {
-                spp.unprotect(properties.getProperty(key))
+                spp.unprotect(properties.getProperty(key), ProtectedPropertyContext.defaultContext(key))
                 return false
             } catch (SensitivePropertyProtectionException e) {
                 logger.expected("Caught a malformed value for ${key}")
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java b/nifi-registry/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
index 2bbefc3..92a1a28 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
@@ -17,8 +17,7 @@
 package org.apache.nifi.registry.web.api;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.properties.PropertyProtectionScheme;
-import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
 import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
 import org.apache.nifi.registry.SecureLdapTestApiApplication;
 import org.apache.nifi.registry.authorization.AccessPolicy;
@@ -130,7 +129,7 @@ public class SecureLdapIT extends IntegrationTestBase {
                 authorizerFactory = new AuthorizerFactory(
                         properties,
                         extensionManager,
-                        sensitivePropertyProvider(),
+                        sensitivePropertyProviderFactory(),
                         registryService,
                         dataSource,
                         identityMapper);
@@ -140,9 +139,8 @@ public class SecureLdapIT extends IntegrationTestBase {
 
         @Primary
         @Bean
-        public static SensitivePropertyProvider sensitivePropertyProvider() throws Exception {
-            return StandardSensitivePropertyProviderFactory.withKey(getNiFiRegistryMasterKeyProvider().getKey())
-                    .getProvider(PropertyProtectionScheme.AES_GCM);
+        public static SensitivePropertyProviderFactory sensitivePropertyProviderFactory() throws Exception {
+            return StandardSensitivePropertyProviderFactory.withKey(getNiFiRegistryMasterKeyProvider().getKey());
         }
 
         private static CryptoKeyProvider getNiFiRegistryMasterKeyProvider() {
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy
index 27cf192..4fdfbf0 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy
@@ -767,7 +767,9 @@ class ConfigEncryptionTool {
         try {
             def doc = getXmlSlurper().parseText(encryptedXml)
             // Find the provider element by class even if it has been renamed
-            def passwords = doc.provider.find { it.'class' as String == LDAP_PROVIDER_CLASS }.property.findAll {
+            def provider = doc.provider.find { it.'class' as String == LDAP_PROVIDER_CLASS }
+            String groupIdentifier = provider.identifier.text()
+            def passwords = provider.property.findAll {
                 it.@name =~ "Password" && it.@encryption != ""
             }
 
@@ -784,7 +786,8 @@ class ConfigEncryptionTool {
                 if (isVerbose) {
                     logger.info("Attempting to decrypt ${password.text()} using protection scheme ${password.@encryption}")
                 }
-                String decryptedValue = sensitivePropertyProvider.unprotect(password.text().trim())
+                final ProtectedPropertyContext context = getContext(providerFactory, (String) password.@name, groupIdentifier)
+                String decryptedValue = sensitivePropertyProvider.unprotect((String) password.text().trim(), context)
                 password.replaceNode {
                     property(name: password.@name, encryption: "none", decryptedValue)
                 }
@@ -806,9 +809,11 @@ class ConfigEncryptionTool {
             def filename = "authorizers.xml"
             def doc = getXmlSlurper().parseText(encryptedXml)
             // Find the provider element by class even if it has been renamed
-            def passwords = doc.userGroupProvider.find {
+            def userGroupProvider = doc.userGroupProvider.find {
                 it.'class' as String == LDAP_USER_GROUP_PROVIDER_CLASS
-            }.property.findAll {
+            }
+            String groupIdentifier = userGroupProvider.identifier.text()
+            def passwords = userGroupProvider.property.findAll {
                 it.@name =~ "Password" && it.@encryption != ""
             }
 
@@ -826,7 +831,8 @@ class ConfigEncryptionTool {
                 }
                 final SensitivePropertyProvider sensitivePropertyProvider = providerFactory
                         .getProvider(PropertyProtectionScheme.fromIdentifier((String) password.@encryption))
-                String decryptedValue = sensitivePropertyProvider.unprotect(password.text().trim())
+                final ProtectedPropertyContext context = getContext(providerFactory, (String) password.@name, groupIdentifier)
+                String decryptedValue = sensitivePropertyProvider.unprotect((String) password.text().trim(), context)
                 password.replaceNode {
                     property(name: password.@name, encryption: "none", decryptedValue)
                 }
@@ -843,6 +849,10 @@ class ConfigEncryptionTool {
         }
     }
 
+    ProtectedPropertyContext getContext(final SensitivePropertyProviderFactory providerFactory, final String propertyName, final String groupIdentifier) {
+        providerFactory.getPropertyContext(groupIdentifier, propertyName);
+    }
+
     String encryptLoginIdentityProviders(final String plainXml, final String newKeyHex = keyHex, final PropertyProtectionScheme newProtectionScheme = protectionScheme) {
         final SensitivePropertyProviderFactory providerFactory = getSensitivePropertyProviderFactory(newKeyHex)
 
@@ -850,9 +860,9 @@ class ConfigEncryptionTool {
         try {
             def doc = getXmlSlurper().parseText(plainXml)
             // Find the provider element by class even if it has been renamed
-            def passwords = doc.provider.find { it.'class' as String == LDAP_PROVIDER_CLASS }
-                    .property.findAll {
-                // Only operate on un-encrypted passwords
+            def provider = doc.provider.find { it.'class' as String == LDAP_PROVIDER_CLASS }
+            String groupIdentifier = provider.identifier.text()
+            def passwords = provider.property.findAll {
                 it.@name =~ "Password" && (it.@encryption == "none" || it.@encryption == "") && it.text()
             }
 
@@ -868,7 +878,8 @@ class ConfigEncryptionTool {
                 if (isVerbose) {
                     logger.info("Attempting to encrypt ${password.name()} using protection scheme ${sensitivePropertyProvider.identifierKey}")
                 }
-                String encryptedValue = sensitivePropertyProvider.protect(password.text().trim())
+                final ProtectedPropertyContext context = getContext(providerFactory, (String) password.@name, groupIdentifier)
+                String encryptedValue = sensitivePropertyProvider.protect((String) password.text().trim(), context)
                 password.replaceNode {
                     property(name: password.@name, encryption: sensitivePropertyProvider.identifierKey, encryptedValue)
                 }
@@ -893,10 +904,13 @@ class ConfigEncryptionTool {
         try {
             def filename = "authorizers.xml"
             def doc = getXmlSlurper().parseText(plainXml)
+
             // Find the provider element by class even if it has been renamed
-            def passwords = doc.userGroupProvider.find { it.'class' as String == LDAP_USER_GROUP_PROVIDER_CLASS }
-                    .property.findAll {
-                // Only operate on un-encrypted passwords
+            def userGroupProvider = doc.userGroupProvider.find {
+                it.'class' as String == LDAP_USER_GROUP_PROVIDER_CLASS
+            }
+            String groupIdentifier = userGroupProvider.identifier.text()
+            def passwords = userGroupProvider.property.findAll {
                 it.@name =~ "Password" && (it.@encryption == "none" || it.@encryption == "") && it.text()
             }
 
@@ -912,7 +926,8 @@ class ConfigEncryptionTool {
                 if (isVerbose) {
                     logger.info("Attempting to encrypt ${password.name()} using protection scheme ${sensitivePropertyProvider.identifierKey}")
                 }
-                String encryptedValue = sensitivePropertyProvider.protect(password.text().trim())
+                final ProtectedPropertyContext context = getContext(providerFactory, (String) password.@name, groupIdentifier)
+                String encryptedValue = sensitivePropertyProvider.protect((String) password.text().trim(), context)
                 password.replaceNode {
                     property(name: password.@name, encryption: sensitivePropertyProvider.identifierKey, encryptedValue)
                 }
@@ -965,7 +980,7 @@ class ConfigEncryptionTool {
             if (!plainProperties.getProperty(key)) {
                 logger.debug("Skipping encryption of ${key} because it is empty")
             } else {
-                String protectedValue = spp.protect(plainProperties.getProperty(key))
+                String protectedValue = spp.protect(plainProperties.getProperty(key), ProtectedPropertyContext.defaultContext(key))
 
                 // Add the encrypted value
                 encryptedProperties.setProperty(key, protectedValue)
@@ -1531,7 +1546,7 @@ class ConfigEncryptionTool {
             if (handlingNiFiProperties || existingNiFiPropertiesAreEncrypted) {
                 final SensitivePropertyProviderFactory sensitivePropertyProviderFactory = getSensitivePropertyProviderFactory(keyHex)
                 SensitivePropertyProvider spp = sensitivePropertyProviderFactory.getProvider(protectionScheme)
-                String encryptedSPK = spp.protect(newFlowPassword)
+                String encryptedSPK = spp.protect(newFlowPassword, ProtectedPropertyContext.defaultContext(NiFiProperties.SENSITIVE_PROPS_KEY))
                 rawProperties.put(NiFiProperties.SENSITIVE_PROPS_KEY, encryptedSPK)
                 // Manually update the protection scheme or it will be lost
                 rawProperties.put(ApplicationPropertiesProtector.getProtectionKey(NiFiProperties.SENSITIVE_PROPS_KEY), spp.getIdentifierKey())
@@ -1568,7 +1583,8 @@ class ConfigEncryptionTool {
                 try {
                     NiFiBootstrapUtils.loadBootstrapProperties(bootstrapConfPath)
                 } catch (final IOException e) {
-                    throw new SensitivePropertyProtectionException(e.getCause(), e)
+                    logger.warn("Could not load default bootstrap.conf: " + e.getMessage())
+                    return BootstrapProperties.EMPTY
                 }
             }
         }
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/DecryptMode.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/DecryptMode.groovy
index 52289c7..1dc010a 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/DecryptMode.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/DecryptMode.groovy
@@ -22,6 +22,7 @@ import org.apache.commons.cli.HelpFormatter
 import org.apache.nifi.properties.ConfigEncryptionTool
 import org.apache.nifi.properties.PropertyProtectionScheme
 import org.apache.nifi.properties.SensitivePropertyProvider
+import org.apache.nifi.properties.SensitivePropertyProviderFactory
 import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory
 import org.apache.nifi.toolkit.encryptconfig.util.BootstrapUtil
 import org.apache.nifi.toolkit.encryptconfig.util.PropertiesEncryptor
@@ -125,7 +126,7 @@ class DecryptMode implements ToolMode {
                 break
 
             case FileType.xml:
-                XmlEncryptor xmlEncryptor = new XmlEncryptor(null, config.decryptionProvider) {
+                XmlEncryptor xmlEncryptor = new XmlEncryptor(null, config.decryptionProvider, config.providerFactory) {
                     @Override
                     List<String> serializeXmlContentAndPreserveFormat(String updatedXmlContent, String originalXmlContent) {
                         // For decrypting unknown, generic XML, this tool will not support preserving the format
@@ -213,6 +214,7 @@ class DecryptMode implements ToolMode {
         PropertyProtectionScheme protectionScheme = ConfigEncryptionTool.DEFAULT_PROTECTION_SCHEME
         String key
         SensitivePropertyProvider decryptionProvider
+        SensitivePropertyProviderFactory providerFactory
         String inputBootstrapPath
 
         FileType fileType
@@ -239,9 +241,9 @@ class DecryptMode implements ToolMode {
                     throw new RuntimeException("Failed to configure tool, could not determine key.")
                 }
             }
-            decryptionProvider = StandardSensitivePropertyProviderFactory
+            providerFactory = StandardSensitivePropertyProviderFactory
                     .withKeyAndBootstrapSupplier(key, ConfigEncryptionTool.getBootstrapSupplier(inputBootstrapPath))
-                    .getProvider(protectionScheme)
+            decryptionProvider = providerFactory.getProvider(protectionScheme)
 
             if (rawOptions.t) {
                 fileType = FileType.valueOf(rawOptions.t)
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryMode.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryMode.groovy
index ff36b22..bb174b0 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryMode.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryMode.groovy
@@ -23,8 +23,10 @@ import org.apache.commons.cli.Options
 import org.apache.nifi.properties.BootstrapProperties
 import org.apache.nifi.properties.ConfigEncryptionTool
 import org.apache.nifi.properties.PropertyProtectionScheme
+import org.apache.nifi.properties.ProtectedPropertyContext
 import org.apache.nifi.properties.SensitivePropertyProtectionException
 import org.apache.nifi.properties.SensitivePropertyProvider
+import org.apache.nifi.properties.SensitivePropertyProviderFactory
 import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory
 import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapUtils
 import org.apache.nifi.toolkit.encryptconfig.util.BootstrapUtil
@@ -284,6 +286,7 @@ class NiFiRegistryMode implements ToolMode {
 
         SensitivePropertyProvider encryptionProvider
         SensitivePropertyProvider decryptionProvider
+        SensitivePropertyProviderFactory providerFactory
 
         boolean writingKeyToBootstrap = false
         String inputBootstrapPath
@@ -335,13 +338,11 @@ class NiFiRegistryMode implements ToolMode {
             if (!encryptionKey) {
                 throw new RuntimeException("Failed to configure tool, could not determine encryption key. Must provide -p, -k, or -b. If using -b, bootstrap.conf argument must already contain root key.")
             }
-            encryptionProvider = StandardSensitivePropertyProviderFactory
+            providerFactory = StandardSensitivePropertyProviderFactory
                     .withKeyAndBootstrapSupplier(encryptionKey, getBootstrapSupplier(inputBootstrapPath))
-                    .getProvider(protectionScheme)
+            encryptionProvider = providerFactory.getProvider(protectionScheme)
 
-            decryptionProvider = decryptionKey ? StandardSensitivePropertyProviderFactory
-                    .withKeyAndBootstrapSupplier(decryptionKey, getBootstrapSupplier(inputBootstrapPath))
-                    .getProvider(oldProtectionScheme) : null
+            decryptionProvider = decryptionKey ? providerFactory.getProvider(oldProtectionScheme) : null
 
             if (handlingNiFiRegistryProperties) {
                 propertiesEncryptor = new NiFiRegistryPropertiesEncryptor(encryptionProvider, decryptionProvider)
@@ -356,14 +357,14 @@ class NiFiRegistryMode implements ToolMode {
             if (handlingIdentityProviders) {
                 inputIdentityProvidersPath = rawOptions.i
                 outputIdentityProvidersPath = rawOptions.I ?: inputIdentityProvidersPath
-                identityProvidersXmlEncryptor = new NiFiRegistryIdentityProvidersXmlEncryptor(encryptionProvider, decryptionProvider)
+                identityProvidersXmlEncryptor = new NiFiRegistryIdentityProvidersXmlEncryptor(encryptionProvider, decryptionProvider, providerFactory)
             }
 
             handlingAuthorizers = rawOptions.a
             if (handlingAuthorizers) {
                 inputAuthorizersPath = rawOptions.a
                 outputAuthorizersPath = rawOptions.A ?: inputAuthorizersPath
-                authorizersXmlEncryptor = new NiFiRegistryAuthorizersXmlEncryptor(encryptionProvider, decryptionProvider)
+                authorizersXmlEncryptor = new NiFiRegistryAuthorizersXmlEncryptor(encryptionProvider, decryptionProvider, providerFactory)
             }
 
         }
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiPropertiesEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiPropertiesEncryptor.groovy
index 28c9ee0..0e1f84e 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiPropertiesEncryptor.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiPropertiesEncryptor.groovy
@@ -34,13 +34,12 @@ class NiFiPropertiesEncryptor extends PropertiesEncryptor {
         super(encryptionProvider, decryptionProvider)
     }
 
-    @Override
-    Properties encrypt(Properties properties) {
+    Properties encrypt(final Properties properties) {
         Set<String> propertiesToEncrypt = new HashSet<>()
         propertiesToEncrypt.addAll(DEFAULT_SENSITIVE_PROPERTIES)
         propertiesToEncrypt.addAll(getAdditionalSensitivePropertyKeys(properties))
 
-        return encrypt(properties, propertiesToEncrypt)
+        encrypt(properties, propertiesToEncrypt)
     }
 
     private static String[] getAdditionalSensitivePropertyKeys(Properties properties) {
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy
index 30639a8..b474e4e 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy
@@ -18,6 +18,7 @@ package org.apache.nifi.toolkit.encryptconfig.util
 
 import groovy.xml.XmlUtil
 import org.apache.nifi.properties.SensitivePropertyProvider
+import org.apache.nifi.properties.SensitivePropertyProviderFactory
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 import org.xml.sax.SAXException
@@ -42,8 +43,9 @@ class NiFiRegistryAuthorizersXmlEncryptor extends XmlEncryptor {
      *   .*?</userGroupProvider>          -> find everything as needed up until and including occurrence of '</userGroupProvider>'
      */
 
-    NiFiRegistryAuthorizersXmlEncryptor(final SensitivePropertyProvider encryptionProvider, final SensitivePropertyProvider decryptionProvider) {
-        super(encryptionProvider, decryptionProvider)
+    NiFiRegistryAuthorizersXmlEncryptor(final SensitivePropertyProvider encryptionProvider, final SensitivePropertyProvider decryptionProvider,
+            final SensitivePropertyProviderFactory providerFactory) {
+        super(encryptionProvider, decryptionProvider, providerFactory)
     }
 
     /**
@@ -55,10 +57,11 @@ class NiFiRegistryAuthorizersXmlEncryptor extends XmlEncryptor {
      * is invoked to encrypt them.
      *
      * @param plainXmlContent the plaintext content of an authorizers.xml file
+     * @param propertyLocation The property location
      * @return the comment with sensitive values encrypted and marked with the cipher.
      */
     @Override
-    String encrypt(String plainXmlContent) {
+    String encrypt(final String plainXmlContent) {
         // First, mark the XML nodes to encrypt that are specific to authorizers.xml by adding an attribute encryption="none"
         String markedXmlContent = markXmlNodesForEncryption(plainXmlContent, "userGroupProvider", {
             it.find {
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy
index fa6ce65..e37788c 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy
@@ -18,6 +18,7 @@ package org.apache.nifi.toolkit.encryptconfig.util
 
 import groovy.xml.XmlUtil
 import org.apache.nifi.properties.SensitivePropertyProvider
+import org.apache.nifi.properties.SensitivePropertyProviderFactory
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 import org.xml.sax.SAXException
@@ -41,8 +42,9 @@ class NiFiRegistryIdentityProvidersXmlEncryptor extends XmlEncryptor {
      *   .*?</provider>                   -> find everything as needed up until and including occurrence of `</provider>`
      */
 
-    NiFiRegistryIdentityProvidersXmlEncryptor(SensitivePropertyProvider encryptionProvider, SensitivePropertyProvider decryptionProvider) {
-        super(encryptionProvider, decryptionProvider)
+    NiFiRegistryIdentityProvidersXmlEncryptor(final SensitivePropertyProvider encryptionProvider, final SensitivePropertyProvider decryptionProvider,
+            final SensitivePropertyProviderFactory providerFactory) {
+        super(encryptionProvider, decryptionProvider, providerFactory)
     }
 
     /**
@@ -54,10 +56,11 @@ class NiFiRegistryIdentityProvidersXmlEncryptor extends XmlEncryptor {
      * is invoked to encrypt them.
      *
      * @param plainXmlContent the plaintext content of an identity-providers.xml file
+     * @param propertyLocation The property location
      * @return the comment with sensitive values encrypted and marked with the cipher.
      */
     @Override
-    String encrypt(String plainXmlContent) {
+    String encrypt(final String plainXmlContent) {
         // First, mark the XML nodes to encrypt that are specific to authorizers.xml by adding an attribute encryption="none"
         String markedXmlContent = markXmlNodesForEncryption(plainXmlContent, "provider", {
             it.find {
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy
index 5ea8d7b..df448aa 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.toolkit.encryptconfig.util
 
+
 import org.apache.nifi.properties.SensitivePropertyProvider
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
@@ -45,13 +46,12 @@ class NiFiRegistryPropertiesEncryptor extends PropertiesEncryptor {
         super(encryptionProvider, decryptionProvider)
     }
 
-    @Override
-    Properties encrypt(Properties properties) {
+    Properties encrypt(final Properties properties) {
         Set<String> propertiesToEncrypt = new HashSet<>()
         propertiesToEncrypt.addAll(DEFAULT_SENSITIVE_PROPERTIES)
         propertiesToEncrypt.addAll(getAdditionalSensitivePropertyKeys(properties))
 
-        return encrypt(properties, propertiesToEncrypt)
+        super.encrypt(properties, propertiesToEncrypt)
     }
 
     private static String[] getAdditionalSensitivePropertyKeys(Properties properties) {
@@ -59,7 +59,7 @@ class NiFiRegistryPropertiesEncryptor extends PropertiesEncryptor {
         if (!rawAdditionalSensitivePropertyKeys) {
             return []
         }
-        return rawAdditionalSensitivePropertyKeys.split(Pattern.quote(","))
+        rawAdditionalSensitivePropertyKeys.split(Pattern.quote(","))
     }
 
 }
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy
index 189e9a5..d43f826 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy
@@ -20,6 +20,7 @@ import groovy.io.GroovyPrintWriter
 import org.apache.commons.configuration2.PropertiesConfiguration
 import org.apache.commons.configuration2.PropertiesConfigurationLayout
 import org.apache.commons.configuration2.builder.fluent.Configurations
+import org.apache.nifi.properties.ProtectedPropertyContext
 import org.apache.nifi.properties.SensitivePropertyProvider
 import org.apache.nifi.util.StringUtils
 import org.slf4j.Logger
@@ -107,7 +108,7 @@ class PropertiesEncryptor {
                 continue
             }
             if (propertiesToDecrypt.keySet().contains(propertyName)) {
-                String decryptedPropertyValue = decryptionProvider.unprotect(propertyValue)
+                String decryptedPropertyValue = decryptionProvider.unprotect(propertyValue, ProtectedPropertyContext.defaultContext(propertyName))
                 unprotectedProperties.setProperty(propertyName, decryptedPropertyValue)
             } else {
                 unprotectedProperties.setProperty(propertyName, propertyValue)
@@ -135,7 +136,7 @@ class PropertiesEncryptor {
             String propertyValue = properties.getProperty(propertyName)
             // empty properties are not encrypted
             if (!StringUtils.isEmpty(propertyValue) && propertiesToEncrypt.contains(propertyName)) {
-                String encryptedPropertyValue = encryptionProvider.protect(propertyValue)
+                String encryptedPropertyValue = encryptionProvider.protect(propertyValue, ProtectedPropertyContext.defaultContext(propertyName))
                 protectedProperties.setProperty(propertyName, encryptedPropertyValue)
                 protectedProperties.setProperty(protectionPropertyForProperty(propertyName), encryptionProvider.getIdentifierKey())
             } else {
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy
index 8324682..c35f129 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy
@@ -19,6 +19,7 @@ package org.apache.nifi.toolkit.encryptconfig.util
 import groovy.util.slurpersupport.GPathResult
 import groovy.xml.XmlUtil
 import org.apache.nifi.properties.SensitivePropertyProvider
+import org.apache.nifi.properties.SensitivePropertyProviderFactory
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 
@@ -30,12 +31,15 @@ abstract class XmlEncryptor {
 
     private static final Logger logger = LoggerFactory.getLogger(XmlEncryptor.class)
 
-    protected SensitivePropertyProvider decryptionProvider
-    protected SensitivePropertyProvider encryptionProvider
+    protected final SensitivePropertyProvider decryptionProvider
+    protected final SensitivePropertyProvider encryptionProvider
+    protected final SensitivePropertyProviderFactory providerFactory
 
-    XmlEncryptor(SensitivePropertyProvider encryptionProvider, SensitivePropertyProvider decryptionProvider) {
+    XmlEncryptor(final SensitivePropertyProvider encryptionProvider, final SensitivePropertyProvider decryptionProvider,
+            final SensitivePropertyProviderFactory providerFactory) {
         this.decryptionProvider = decryptionProvider
         this.encryptionProvider = encryptionProvider
+        this.providerFactory = providerFactory
     }
 
     static boolean supportsFile(String filePath) {
@@ -63,7 +67,7 @@ abstract class XmlEncryptor {
         }
     }
 
-    String decrypt(String encryptedXmlContent) {
+    String decrypt(final String encryptedXmlContent) {
         try {
 
             def doc = new XmlSlurper().parseText(encryptedXmlContent)
@@ -90,7 +94,9 @@ abstract class XmlEncryptor {
                             "This tool supports ${supportedDecryptionScheme}, but this xml file contains " +
                             "${node.toString()} protected by ${node.@encryption}")
                 }
-                String decryptedValue = decryptionProvider.unprotect(node.text().trim())
+                String groupIdentifier = (String) node.parent().identifier
+                String propertyName = (String) node.@name
+                String decryptedValue = decryptionProvider.unprotect(node.text().trim(), providerFactory.getPropertyContext(groupIdentifier, propertyName))
                 node.@encryption = ENCRYPTION_NONE
                 node.replaceBody(decryptedValue)
             }
@@ -105,7 +111,7 @@ abstract class XmlEncryptor {
         }
     }
 
-    String encrypt(String plainXmlContent) {
+    String encrypt(final String plainXmlContent) {
         try {
             def doc = new XmlSlurper().parseText(plainXmlContent)
 
@@ -113,14 +119,16 @@ abstract class XmlEncryptor {
                 node.text() && node.@encryption == ENCRYPTION_NONE
             }
 
-            logger.debug("Encrypting ${nodesToEncrypt.size()} element(s) of XML decoument")
+            logger.debug("Encrypting ${nodesToEncrypt.size()} element(s) of XML document")
 
             if (nodesToEncrypt.size() == 0) {
                 return plainXmlContent
             }
 
             nodesToEncrypt.each { node ->
-                String encryptedValue = this.encryptionProvider.protect(node.text().trim())
+                String groupIdentifier = (String) node.parent().identifier
+                String propertyName = (String) node.@name
+                String encryptedValue = this.encryptionProvider.protect(node.text().trim(), providerFactory.getPropertyContext(groupIdentifier, propertyName))
                 node.@encryption = this.encryptionProvider.getIdentifierKey()
                 node.replaceBody(encryptedValue)
             }
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
index 57be63b..580cddc 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
@@ -1726,10 +1726,11 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         ConfigEncryptionTool tool = new ConfigEncryptionTool()
 
         // Sanity check for decryption
+        String propertyName = "Manager Password"
         String cipherText = "q4r7WIgN0MaxdAKM||SGgdCTPGSFEcuH4RraMYEdeyVbOx93abdWTVSWvh1w+klA"
         String EXPECTED_PASSWORD = "thisIsABadPassword"
         final SensitivePropertyProvider spp = StandardSensitivePropertyProviderFactory.withKey(KEY_HEX_128).getProvider(tool.protectionScheme)
-        assert spp.unprotect(cipherText) == EXPECTED_PASSWORD
+        assert spp.unprotect(cipherText, ldapPropertyContext(propertyName)) == EXPECTED_PASSWORD
 
         tool.keyHex = KEY_HEX_128
 
@@ -1840,6 +1841,14 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert decryptedLines == lines
     }
 
+    private static ProtectedPropertyContext ldapPropertyContext(final String propertyName) {
+        ProtectedPropertyContext.contextFor("ldap", propertyName)
+    }
+
+    private static ProtectedPropertyContext nifiPropertiesContext(final String propertyName) {
+        ProtectedPropertyContext.defaultContext(propertyName)
+    }
+
     @Test
     void testShouldEncryptLoginIdentityProviders() {
         // Arrange
@@ -1873,8 +1882,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert passwordLines.every { it.contains(encryptionScheme) }
         passwordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -1912,8 +1922,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert populatedPasswordLines.every { it.contains(encryptionScheme) }
         populatedPasswordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -1950,8 +1961,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert passwordLines.every { it.contains(encryptionScheme) }
         passwordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -1988,8 +2000,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert passwordLines.every { it.contains(encryptionScheme) }
         passwordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -2028,8 +2041,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert populatedPasswordLines.every { it.contains(encryptionScheme) }
         populatedPasswordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -2364,7 +2378,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 }
 
                 encryptedValues.each {
-                    assert spp.unprotect(it.text()) == PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ldapPropertyContext((String) it.@name)) == PASSWORD
                 }
 
                 // Check that the key was persisted to the bootstrap.conf
@@ -2446,7 +2460,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 }
 
                 encryptedValues.each {
-                    assert spp.unprotect(it.text()) == PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ldapPropertyContext((String) it.@name)) == PASSWORD
                 }
 
                 // Check that the key was persisted to the bootstrap.conf
@@ -2479,6 +2493,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
     void testShouldDecryptAuthorizers() {
         // Arrange
         String authorizersPath = "src/test/resources/authorizers-populated-encrypted.xml"
+        String propertyName = "Manager Password"
         File authorizersFile = new File(authorizersPath)
 
         setupTmpDir()
@@ -2492,7 +2507,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         String cipherText = "q4r7WIgN0MaxdAKM||SGgdCTPGSFEcuH4RraMYEdeyVbOx93abdWTVSWvh1w+klA"
         String EXPECTED_PASSWORD = "thisIsABadPassword"
         final SensitivePropertyProvider spp = StandardSensitivePropertyProviderFactory.withKey(KEY_HEX_128).getProvider(tool.protectionScheme)
-        assert spp.unprotect(cipherText) == EXPECTED_PASSWORD
+        assert spp.unprotect(cipherText, ldapPropertyContext(propertyName)) == EXPECTED_PASSWORD
 
         tool.keyHex = KEY_HEX_128
 
@@ -2636,8 +2651,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert passwordLines.every { it.contains(encryptionScheme) }
         passwordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -2675,8 +2691,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert populatedPasswordLines.every { it.contains(encryptionScheme) }
         populatedPasswordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -2713,8 +2730,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert passwordLines.every { it.contains(encryptionScheme) }
         passwordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -2751,8 +2769,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert passwordLines.every { it.contains(encryptionScheme) }
         passwordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -2791,8 +2810,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert populatedPasswordLines.every { it.contains(encryptionScheme) }
         populatedPasswordLines.each {
             String ct = (it =~ ">(.*)</property>")[0][1]
+            String propertyName = (it =~ 'name="(.*)"')[0][1]
             logger.info("Cipher text: ${ct}")
-            assert spp.unprotect(ct) == PASSWORD
+            assert spp.unprotect(ct, ldapPropertyContext(propertyName)) == PASSWORD
         }
     }
 
@@ -3091,7 +3111,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 }
 
                 encryptedValues.each {
-                    assert spp.unprotect(it.text()) == PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ldapPropertyContext((String) it.@name)) == PASSWORD
                 }
 
                 // Check that the key was persisted to the bootstrap.conf
@@ -3172,7 +3192,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 }
 
                 encryptedValues.each {
-                    assert spp.unprotect(it.text()) == PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ldapPropertyContext((String) it.@name)) == PASSWORD
                 }
 
                 // Check that the key was persisted to the bootstrap.conf
@@ -3253,7 +3273,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 }
 
                 encryptedValues.each {
-                    assert spp.unprotect(it.text()) == PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ldapPropertyContext((String) it.@name)) == PASSWORD
                 }
 
                 // Check that the key was persisted to the bootstrap.conf
@@ -3381,7 +3401,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                     it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
                 }
                 lipEncryptedValues.each {
-                    assert spp.unprotect(it.text()) == PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ldapPropertyContext((String) it.@name)) == PASSWORD
                 }
                 // Check that the comments are still there
                 def lipTrimmedLines = inputLIPFile.readLines().collect { it.trim() }.findAll { it }
@@ -3407,7 +3427,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                     it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
                 }
                 authorizersEncryptedValues.each {
-                    assert spp.unprotect(it.text()) == PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ldapPropertyContext((String) it.@name)) == PASSWORD
                 }
                 // Check that the comments are still there
                 def authorizersTrimmedLines = inputAuthorizersFile.readLines().collect { it.trim() }.findAll { it }
@@ -3786,7 +3806,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 assert newSensitivePropertyKey != originalSensitiveValues.get(NiFiProperties.SENSITIVE_PROPS_KEY)
 
                 // Check that the decrypted value is the new password
-                assert spp.unprotect(newSensitivePropertyKey) == newFlowPassword
+                assert spp.unprotect(newSensitivePropertyKey, nifiPropertiesContext(NiFiProperties.SENSITIVE_PROPS_KEY)) == newFlowPassword
 
                 // Check that all other values stayed the same
                 originalEncryptedValues.every { String key, String originalValue ->
@@ -3798,7 +3818,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 // Check that all other (decrypted) values stayed the same
                 originalSensitiveValues.every { String key, String originalValue ->
                     if (key != NiFiProperties.SENSITIVE_PROPS_KEY) {
-                        assert spp.unprotect(updatedProperties.getProperty(key)) == originalValue
+                        assert spp.unprotect(updatedProperties.getProperty(key), nifiPropertiesContext(key)) == originalValue
                     }
                 }
 
@@ -3919,7 +3939,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 assert newSensitivePropertyKey != originalSensitiveValues.get(NiFiProperties.SENSITIVE_PROPS_KEY)
 
                 // Check that the decrypted value is the new password
-                assert spp.unprotect(newSensitivePropertyKey) == newFlowPassword
+                assert spp.unprotect(newSensitivePropertyKey, nifiPropertiesContext(NiFiProperties.SENSITIVE_PROPS_KEY)) == newFlowPassword
 
                 // Check that all other values stayed the same
                 originalEncryptedValues.every { String key, String originalValue ->
@@ -3931,7 +3951,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 // Check that all other (decrypted) values stayed the same
                 originalSensitiveValues.every { String key, String originalValue ->
                     if (key != NiFiProperties.SENSITIVE_PROPS_KEY) {
-                        assert spp.unprotect(updatedProperties.getProperty(key)) == originalValue
+                        assert spp.unprotect(updatedProperties.getProperty(key), nifiPropertiesContext(key)) == originalValue
                     }
                 }
 
@@ -4071,7 +4091,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 assert newSensitivePropertyKey != originalSensitiveValues.get(NiFiProperties.SENSITIVE_PROPS_KEY)
 
                 // Check that the decrypted value is the new password
-                assert spp.unprotect(newSensitivePropertyKey) == newFlowPassword
+                assert spp.unprotect(newSensitivePropertyKey, nifiPropertiesContext(NiFiProperties.SENSITIVE_PROPS_KEY)) == newFlowPassword
 
                 // Check that all other values stayed the same
                 originalEncryptedValues.every { String key, String originalValue ->
@@ -4083,7 +4103,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 // Check that all other (decrypted) values stayed the same
                 originalSensitiveValues.every { String key, String originalValue ->
                     if (key != NiFiProperties.SENSITIVE_PROPS_KEY) {
-                        assert spp.unprotect(updatedProperties.getProperty(key)) == originalValue
+                        assert spp.unprotect(updatedProperties.getProperty(key), nifiPropertiesContext(key)) == originalValue
                     }
                 }
 
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/EncryptConfigMainTest.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/EncryptConfigMainTest.groovy
index a33ee26..72c3a0f 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/EncryptConfigMainTest.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/EncryptConfigMainTest.groovy
@@ -19,6 +19,7 @@ package org.apache.nifi.toolkit.encryptconfig
 
 import org.apache.nifi.properties.NiFiPropertiesLoader
 import org.apache.nifi.properties.PropertyProtectionScheme
+import org.apache.nifi.properties.ProtectedPropertyContext
 import org.apache.nifi.properties.SensitivePropertyProvider
 import org.apache.nifi.toolkit.encryptconfig.util.BootstrapUtil
 import org.apache.nifi.util.NiFiProperties
@@ -221,7 +222,7 @@ class EncryptConfigMainTest extends GroovyTestCase {
                     it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
                 }
                 lipEncryptedValues.each {
-                    assert spp.unprotect(it.text()) == TestUtil.PASSWORD
+                    assert spp.unprotect((String) it.text(), ProtectedPropertyContext.defaultContext((String) it.@name)) == TestUtil.PASSWORD
                 }
                 // Check that the comments are still there
                 def lipTrimmedLines = inputLIPFile.readLines().collect { it.trim() }.findAll { it }
@@ -245,7 +246,7 @@ class EncryptConfigMainTest extends GroovyTestCase {
                     it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
                 }
                 authorizersEncryptedValues.each {
-                    assert spp.unprotect(it.text()) == TestUtil.PASSWORD
+                    assert spp.unprotect((String) it.text(), (ProtectedPropertyContext) ProtectedPropertyContext.defaultContext((String) it.@name)) == TestUtil.PASSWORD
                 }
                 // Check that the comments are still there
                 def authorizersTrimmedLines = inputAuthorizersFile.readLines().collect { it.trim() }.findAll { it }
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryDecryptModeSpec.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryDecryptModeSpec.groovy
index 2ff1414..e8f86f4 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryDecryptModeSpec.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryDecryptModeSpec.groovy
@@ -61,7 +61,7 @@ class NiFiRegistryDecryptModeSpec extends Specification {
 
         setup:
         NiFiRegistryDecryptMode tool = new NiFiRegistryDecryptMode()
-        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128)
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128, "nifi-registry.properties")
         File outRegistryProperties1 = generateTmpFile()
 
         when: "run with args: -k <key> -r <file>"
@@ -81,7 +81,7 @@ class NiFiRegistryDecryptModeSpec extends Specification {
 
         setup:
         NiFiRegistryDecryptMode tool = new NiFiRegistryDecryptMode()
-        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_PASSWORD_256)
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_PASSWORD_256, "nifi-registry.properties")
         File outRegistryProperties1 = generateTmpFile()
 
         when: "run with args: -p <password> -r <file>"
@@ -99,7 +99,7 @@ class NiFiRegistryDecryptModeSpec extends Specification {
 
         setup:
         NiFiRegistryDecryptMode tool = new NiFiRegistryDecryptMode()
-        def inRegistryProperties = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128)
+        def inRegistryProperties = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128, "nifi-registry.properties")
         def inBootstrap = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128)
         File outRegistryProperties = generateTmpFile()
 
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryModeSpec.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryModeSpec.groovy
index 137eeea..3d7e87e 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryModeSpec.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryModeSpec.groovy
@@ -168,9 +168,9 @@ class NiFiRegistryModeSpec extends Specification {
         setup:
         NiFiRegistryMode tool = new NiFiRegistryMode()
         def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inAuthorizers1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED)
+        def inAuthorizers1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED, "authorizers.xml")
         def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inAuthorizers2 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED)
+        def inAuthorizers2 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED, "authorizers.xml")
         def outAuthorizers2 = generateTmpFilePath()
 
         when: "run with args: -k <key> -b <file> -a <file>"
@@ -194,9 +194,9 @@ class NiFiRegistryModeSpec extends Specification {
         setup:
         NiFiRegistryMode tool = new NiFiRegistryMode()
         def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inAuthorizersXml1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_COMMENTED)
+        def inAuthorizersXml1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_COMMENTED, "authorizers.xml")
         def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inAuthorizersXml2 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_EMPTY)
+        def inAuthorizersXml2 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_EMPTY, "authorizers.xml")
         def outAuthorizers2 = generateTmpFilePath()
 
         when: "run with args: -k <key> -b <file> -a <file_with_no_sensitive_props>"
@@ -221,9 +221,9 @@ class NiFiRegistryModeSpec extends Specification {
         setup:
         NiFiRegistryMode tool = new NiFiRegistryMode()
         def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED)
+        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED, "login-identity-providers.xml")
         def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inIdentityProviders2 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED)
+        def inIdentityProviders2 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED, "login-identity-providers.xml")
         def outIdentityProviders2 = generateTmpFilePath()
 
         when: "run with args: -k <key> -b <file> -i <file>"
@@ -247,9 +247,9 @@ class NiFiRegistryModeSpec extends Specification {
         setup:
         NiFiRegistryMode tool = new NiFiRegistryMode()
         def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_COMMENTED)
+        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_COMMENTED, "login-identity-providers.xml")
         def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inIdentityProviders2 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_EMPTY)
+        def inIdentityProviders2 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_EMPTY, "login-identity-providers.xml")
         def outIdentityProviders2 = generateTmpFilePath()
 
         when: "run with args: -k <key> -b <file> -i <file_with_no_sensitive_props>"
@@ -274,9 +274,9 @@ class NiFiRegistryModeSpec extends Specification {
         setup:
         NiFiRegistryMode tool = new NiFiRegistryMode()
         def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
-        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED)
-        def inAuthorizers1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED)
-        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED)
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED, "nifi-registry.properties")
+        def inAuthorizers1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED, "authorizers")
+        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED, "login-identity-providers.xml")
 
         when: "run with args: -k <key> -b <file> -r <file> -a <file_with_no_sensitive_props> -i <file_with_no_sensitive_props>"
         tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -r ${inRegistryProperties1} -a ${inAuthorizers1} -i ${inIdentityProviders1}".split(" "))
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy
index 4c9678e..12f6e3a 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy
@@ -101,8 +101,12 @@ class TestUtil {
     }
 
     static String generateTmpFilePath() {
+        generateTmpFilePath("tmp_file")
+    }
+
+    static String generateTmpFilePath(final String tempFileSuffix) {
         File tmpDir = setupTmpDir()
-        return "${tmpDir.getAbsolutePath()}/${UUID.randomUUID().toString()}.tmp_file"
+        return "${tmpDir.getAbsolutePath()}/${UUID.randomUUID().toString()}.${tempFileSuffix}"
     }
 
     static File generateTmpFile() {
@@ -110,8 +114,17 @@ class TestUtil {
         tmpFile
     }
 
+    static File generateTmpFile(final String tempFileSuffix) {
+        File tmpFile = new File(generateTmpFilePath(tempFileSuffix))
+        tmpFile
+    }
+
     static String copyFileToTempFile(String filePath) {
-        File tmpFile = generateTmpFile()
+        copyFileToTempFile(filePath, "tmp_file")
+    }
+
+    static String copyFileToTempFile(String filePath, final String tempFileSuffix) {
+        File tmpFile = generateTmpFile(tempFileSuffix)
         tmpFile.text = new File(filePath).text
         return tmpFile.getAbsolutePath()
     }
@@ -306,7 +319,7 @@ class TestUtil {
             String propertyValue = value
             assert it.@encryption == expectedProtectionScheme
             assert !plaintextValues.contains(propertyValue)
-            assert plaintextValues.contains(spp.unprotect(propertyValue))
+            assert plaintextValues.contains(spp.unprotect(propertyValue, org.apache.nifi.properties.ProtectedPropertyContext.defaultContext((String) it.@name)))
         }
 
         return true