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/03/03 03:39:09 UTC

[nifi] branch main updated: NIFI-7127 - Allow choosing a SecureHasher for Flow Fingerprint checking

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 c5abf2b  NIFI-7127 - Allow choosing a SecureHasher for Flow Fingerprint checking
c5abf2b is described below

commit c5abf2ba5443cf952eccb0e00d0858de49511f4f
Author: Nathan Gough <th...@gmail.com>
AuthorDate: Tue Mar 2 12:14:59 2021 -0500

    NIFI-7127 - Allow choosing a SecureHasher for Flow Fingerprint checking
    
    - Configuration based on Sensitive Properties Algorithm defaults to Argon2
    - Added SensitiveValueEncoder interface
    - Standard implementation uses existing approach with HmacSHA256
    
    This closes #4867
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../nifi/security/util/crypto/CipherUtility.java   | 55 +++---------
 .../util/crypto/SecureHasherException.java         | 33 ++++++++
 .../security/util/crypto/SecureHasherFactory.java  | 59 +++++++++++++
 .../util/crypto/SecureHasherFactoryTest.java       | 97 ++++++++++++++++++++++
 .../flow/PopularVoteFlowElectionFactoryBean.java   |  9 +-
 .../PopularVoteFlowElectionFactoryBeanTest.groovy  | 24 ------
 .../flow/TestPopularVoteFlowElection.java          | 12 ++-
 .../apache/nifi/cluster/integration/Cluster.java   |  5 +-
 .../apache/nifi/encrypt/SensitiveValueEncoder.java | 29 +++++++
 .../encrypt/StandardSensitiveValueEncoder.java     | 81 ++++++++++++++++++
 .../org/apache/nifi/controller/FlowController.java | 13 +++
 .../inheritance/FlowFingerprintCheck.java          |  4 +-
 .../nifi/fingerprint/FingerprintFactory.java       | 76 +++++------------
 .../FingerprintFactoryGroovyTest.groovy            |  5 +-
 .../nifi/fingerprint/FingerprintFactoryTest.java   | 72 +++++++++-------
 15 files changed, 416 insertions(+), 158 deletions(-)

diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java
index 46668ad..85016ec 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/CipherUtility.java
@@ -16,6 +16,16 @@
  */
 package org.apache.nifi.security.util.crypto;
 
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.EncryptionMethod;
+import org.apache.nifi.security.util.KeyDerivationFunction;
+import org.apache.nifi.stream.io.ByteCountingInputStream;
+import org.apache.nifi.stream.io.ByteCountingOutputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+
+import javax.crypto.Cipher;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -33,15 +43,6 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-import javax.crypto.Cipher;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.security.util.EncryptionMethod;
-import org.apache.nifi.security.util.KeyDerivationFunction;
-import org.apache.nifi.stream.io.ByteCountingInputStream;
-import org.apache.nifi.stream.io.ByteCountingOutputStream;
-import org.apache.nifi.stream.io.StreamUtils;
 
 public class CipherUtility {
 
@@ -350,42 +351,6 @@ public class CipherUtility {
     }
 
     /**
-     * Returns a securely-derived, deterministic value from the provided plaintext property
-     * value. This is because sensitive values should not be disclosed through the
-     * logs. However, the equality or difference of the sensitive value can be important, so it cannot be ignored completely.
-     *
-     * The specific derivation process is unimportant as long as it is a salted,
-     * cryptographically-secure hash function with an iteration cost sufficient for password
-     * storage in other applications.
-     *
-     * @param sensitivePropertyValue the plaintext property value
-     * @return a deterministic string value which represents this input but is safe to print in a log
-     */
-    public static String getLoggableRepresentationOfSensitiveValue(String sensitivePropertyValue) {
-        // TODO: Use DI/IoC to inject this implementation in the constructor of the FingerprintFactory
-        // There is little initialization cost, so it doesn't make sense to cache this as a field
-        SecureHasher secureHasher = new Argon2SecureHasher();
-
-        // TODO: Extend with secure hashing capability and inject?
-        return getLoggableRepresentationOfSensitiveValue(sensitivePropertyValue, secureHasher);
-    }
-
-    /**
-     * Returns a securely-derived, deterministic value from the provided plaintext property
-     * value. This is because sensitive values should not be disclosed through the
-     * logs. However, the equality or difference of the sensitive value can be important, so it cannot be ignored completely.
-     *
-     * The specific derivation process is determined by the provided {@link SecureHasher} implementation.
-     *
-     * @param sensitivePropertyValue the plaintext property value
-     * @param secureHasher an instance of {@link SecureHasher} which will be used to mask the value
-     * @return a deterministic string value which represents this input but is safe to print in a log
-     */
-    public static String getLoggableRepresentationOfSensitiveValue(String sensitivePropertyValue, SecureHasher secureHasher) {
-        return "[MASKED] (" + secureHasher.hashBase64(sensitivePropertyValue) + ")";
-    }
-
-    /**
      * Returns the current timestamp in a default format. Used by many encryption operations for logging/debugging.
      *
      * @return the current timestamp in 'yyyy-MM-dd HH:mm:ss.SSS Z' format
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasherException.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasherException.java
new file mode 100644
index 0000000..bdf209e
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasherException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.security.util.crypto;
+
+/**
+ * Exception indicating an error occurred instantiating a SecureHasher.
+ */
+public class SecureHasherException extends RuntimeException {
+
+    private static final long serialVersionUID = 1L;
+
+    public SecureHasherException(String message) {
+        super(message);
+    }
+
+    public SecureHasherException(String message, Throwable cause) {
+        super(message, cause);
+    }
+}
\ No newline at end of file
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasherFactory.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasherFactory.java
new file mode 100644
index 0000000..6b283ee
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasherFactory.java
@@ -0,0 +1,59 @@
+/*
+ * 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.security.util.crypto;
+
+import org.apache.nifi.security.util.KeyDerivationFunction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * <p> Provides a factory for SecureHasher implementations. Will return Argon2 by default if no algorithm parameter is given.
+ * Algorithm parameter should align with the below registered secure hasher names (PBKDF2, BCRYPT, SCRYPT, ARGON2).
+ */
+public class SecureHasherFactory {
+    private static final Logger LOGGER = LoggerFactory.getLogger(SecureHasherFactory.class);
+
+    private static Map<KeyDerivationFunction, Class<? extends SecureHasher>> registeredSecureHashers;
+    private static final Class<? extends SecureHasher> DEFAULT_SECURE_HASHER_CLASS = Argon2SecureHasher.class;
+
+    static {
+        registeredSecureHashers = new HashMap<>();
+        registeredSecureHashers.put(KeyDerivationFunction.PBKDF2, PBKDF2SecureHasher.class);
+        registeredSecureHashers.put(KeyDerivationFunction.BCRYPT, BcryptSecureHasher.class);
+        registeredSecureHashers.put(KeyDerivationFunction.SCRYPT, ScryptSecureHasher.class);
+        registeredSecureHashers.put(KeyDerivationFunction.ARGON2, Argon2SecureHasher.class);
+    }
+
+    public static SecureHasher getSecureHasher(final String algorithm) {
+        Class<? extends SecureHasher> secureHasherClass = DEFAULT_SECURE_HASHER_CLASS;
+        final String algorithmPattern = algorithm.toUpperCase();
+        try {
+            for (KeyDerivationFunction keyDerivationFunction : registeredSecureHashers.keySet()) {
+                final String functionName = keyDerivationFunction.getKdfName().toUpperCase();
+                if (algorithmPattern.contains(functionName)) {
+                    secureHasherClass = registeredSecureHashers.get(keyDerivationFunction);
+                }
+            }
+            LOGGER.debug("Creating SecureHasher [{}] for algorithm [{}]", secureHasherClass.getName(), algorithm);
+            return secureHasherClass.getDeclaredConstructor().newInstance();
+        } catch (Exception e) {
+            throw new SecureHasherException(String.format("Failed to create SecureHasher for algorithm [%s]", algorithm), e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/SecureHasherFactoryTest.java b/nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/SecureHasherFactoryTest.java
new file mode 100644
index 0000000..0f4a6cb
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/java/org/apache/nifi/security/util/crypto/SecureHasherFactoryTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.security.util.crypto;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+public class SecureHasherFactoryTest {
+
+    private static final Argon2SecureHasher DEFAULT_HASHER = new Argon2SecureHasher();
+
+    @Rule
+    public ExpectedException exceptionRule = ExpectedException.none();
+
+    @Test
+    public void testSecureHasherFactoryArgon2() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("NIFI_ARGON2_AES_GCM_128");
+        assertEquals(Argon2SecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryPBKDF2() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("NIFI_PBKDF2_AES_GCM_128");
+        assertEquals(PBKDF2SecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryBCrypt() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("NIFI_BCRYPT_AES_GCM_128");
+        assertEquals(BcryptSecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactorySCrypt() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("NIFI_SCRYPT_AES_GCM_128");
+        assertEquals(ScryptSecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryArgon2ShortName() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("ARGON2");
+        assertEquals(Argon2SecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactorySCryptShortName() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("SCRYPT");
+        assertEquals(ScryptSecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryLowerCaseName() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("scrypt");
+        assertEquals(ScryptSecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryArgon2SimilarName() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("ARGON_2");
+        assertEquals(Argon2SecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryFailsUnknownAlgorithmName() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("wrongString");
+        assertEquals(Argon2SecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryDefaultsToArgon2IfLongUnknownAlgorithmName() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("NIFI_UNKNONWN_AES_GCM_256");
+        assertEquals(Argon2SecureHasher.class, hasher.getClass());
+    }
+
+    @Test
+    public void testSecureHasherFactoryEmptyString() {
+        SecureHasher hasher = SecureHasherFactory.getSecureHasher("");
+        assertEquals(DEFAULT_HASHER.getClass(), hasher.getClass());
+    }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
index be23599..b892e82 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBean.java
@@ -17,10 +17,10 @@
 
 package org.apache.nifi.cluster.coordination.flow;
 
-import java.util.concurrent.TimeUnit;
-
 import org.apache.nifi.encrypt.PropertyEncryptor;
 import org.apache.nifi.encrypt.PropertyEncryptorFactory;
+import org.apache.nifi.encrypt.SensitiveValueEncoder;
+import org.apache.nifi.encrypt.StandardSensitiveValueEncoder;
 import org.apache.nifi.fingerprint.FingerprintFactory;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.util.FormatUtils;
@@ -29,6 +29,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.FactoryBean;
 
+import java.util.concurrent.TimeUnit;
+
 public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVoteFlowElection> {
     private static final Logger logger = LoggerFactory.getLogger(PopularVoteFlowElectionFactoryBean.class);
     private NiFiProperties properties;
@@ -48,7 +50,8 @@ public class PopularVoteFlowElectionFactoryBean implements FactoryBean<PopularVo
 
         final Integer maxNodes = properties.getFlowElectionMaxCandidates();
         final PropertyEncryptor encryptor = PropertyEncryptorFactory.getPropertyEncryptor(properties);
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
+        final SensitiveValueEncoder sensitiveValueEncoder = new StandardSensitiveValueEncoder(properties);
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager, sensitiveValueEncoder);
         return new PopularVoteFlowElection(maxWaitMillis, TimeUnit.MILLISECONDS, maxNodes, fingerprintFactory);
     }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
index d73552c..4e7f7f9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/flow/PopularVoteFlowElectionFactoryBeanTest.groovy
@@ -60,30 +60,6 @@ class PopularVoteFlowElectionFactoryBeanTest extends GroovyTestCase {
     }
 
     @Test
-    void testGetObjectShouldPopulateDefaultSensitivePropsKeyIfEmpty() {
-        // Arrange
-        PopularVoteFlowElectionFactoryBean electionFactoryBean = new PopularVoteFlowElectionFactoryBean()
-        electionFactoryBean.properties = mockProperties()
-
-        final PropertyEncryptor DEFAULT_ENCRYPTOR = PropertyEncryptorFactory.getPropertyEncryptor(mockProperties())
-        final String EXPECTED_PLAINTEXT = "my.test.value"
-        final String EXPECTED_CIPHERTEXT = DEFAULT_ENCRYPTOR.encrypt(EXPECTED_PLAINTEXT)
-        logger.info("Expected ciphertext: ${EXPECTED_CIPHERTEXT}")
-
-        // Act
-        PopularVoteFlowElection election = electionFactoryBean.object
-        logger.info("Got object: ${election}")
-
-        // Assert
-
-        // Violates LoD but need to evaluate nested encryptor can decrypt
-        def encryptor = election.fingerprintFactory.encryptor
-        String decrypted = encryptor.decrypt(EXPECTED_CIPHERTEXT)
-        logger.info("Decrypted plain text: ${decrypted}")
-        assert decrypted == EXPECTED_PLAINTEXT
-    }
-
-    @Test
     void testGetObjectShouldPopulateSensitivePropsKeyIfPresent() {
         // Arrange
         final String REVERSE_KEY = DEFAULT_SENSITIVE_PROPS_KEY.reverse()
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
index 5f287fa..f00a56c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/flow/TestPopularVoteFlowElection.java
@@ -21,6 +21,7 @@ import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.cluster.protocol.StandardDataFlow;
 import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.encrypt.SensitiveValueEncoder;
 import org.apache.nifi.fingerprint.FingerprintFactory;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
@@ -58,6 +59,15 @@ public class TestPopularVoteFlowElection {
         };
     }
 
+    private SensitiveValueEncoder createSensitiveValueEncoder() {
+        return new SensitiveValueEncoder() {
+            @Override
+            public String getEncoded(String encryptedPropertyValue) {
+                return String.format("[MASKED] %s", encryptedPropertyValue);
+            }
+        };
+    }
+
     @Test
     public void testOnlyEmptyFlows() throws IOException {
         final FingerprintFactory fingerprintFactory = Mockito.mock(FingerprintFactory.class);
@@ -154,7 +164,7 @@ public class TestPopularVoteFlowElection {
     @Test
     public void testAutoGeneratedVsPopulatedFlowElection() throws IOException {
         final ExtensionManager extensionManager = new StandardExtensionDiscoveringManager();
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptor(), extensionManager);
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(createEncryptor(), extensionManager, createSensitiveValueEncoder());
         final PopularVoteFlowElection election = new PopularVoteFlowElection(1, TimeUnit.MINUTES, 4, fingerprintFactory);
         final byte[] emptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/auto-generated-empty-flow.xml"));
         final byte[] nonEmptyFlow = Files.readAllBytes(Paths.get("src/test/resources/conf/reporting-task-flow.xml"));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
index 70ac468..00b76f1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/integration/Cluster.java
@@ -27,6 +27,8 @@ import org.apache.nifi.cluster.coordination.flow.PopularVoteFlowElection;
 import org.apache.nifi.cluster.coordination.node.ClusterRoles;
 import org.apache.nifi.encrypt.PropertyEncryptor;
 import org.apache.nifi.encrypt.PropertyEncryptorFactory;
+import org.apache.nifi.encrypt.SensitiveValueEncoder;
+import org.apache.nifi.encrypt.StandardSensitiveValueEncoder;
 import org.apache.nifi.fingerprint.FingerprintFactory;
 import org.apache.nifi.nar.ExtensionDiscoveringManager;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
@@ -135,8 +137,9 @@ public class Cluster {
         final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", addProps);
 
         final PropertyEncryptor encryptor = PropertyEncryptorFactory.getPropertyEncryptor(nifiProperties);
+        final SensitiveValueEncoder sensitiveValueEncoder = new StandardSensitiveValueEncoder(nifiProperties);
         final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager, sensitiveValueEncoder);
         final FlowElection flowElection = new PopularVoteFlowElection(flowElectionTimeoutMillis, TimeUnit.MILLISECONDS, flowElectionMaxNodes, fingerprintFactory);
 
         final Node node = new Node(nifiProperties, extensionManager, flowElection);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/encrypt/SensitiveValueEncoder.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/encrypt/SensitiveValueEncoder.java
new file mode 100644
index 0000000..8dc3186
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/encrypt/SensitiveValueEncoder.java
@@ -0,0 +1,29 @@
+/*
+ * 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.encrypt;
+
+/** SensitiveValueEncoder implementations should securely encode a sensitive value into a loggable, secure representation
+ * of that value.
+ */
+public interface SensitiveValueEncoder {
+
+    /**
+     * Returns a securely-derived, deterministic representation of a decrypted NiFi sensitive property value
+     * for logging/comparison purposes.
+     */
+    String getEncoded(String plaintextSensitiveValue);
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/encrypt/StandardSensitiveValueEncoder.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/encrypt/StandardSensitiveValueEncoder.java
new file mode 100644
index 0000000..3bdf1d3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/encrypt/StandardSensitiveValueEncoder.java
@@ -0,0 +1,81 @@
+/*
+ * 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.encrypt;
+
+import org.apache.nifi.security.util.crypto.SecureHasher;
+import org.apache.nifi.security.util.crypto.SecureHasherFactory;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Base64;
+import java.util.Objects;
+
+/**
+ * Encode a sensitive value using the NiFi sensitive properties key to derive the secret key used in the MAC operation.
+ */
+public class StandardSensitiveValueEncoder implements SensitiveValueEncoder {
+
+    private static final Logger logger = LoggerFactory.getLogger(StandardSensitiveValueEncoder.class);
+
+    private SecretKeySpec secretKeySpec;
+    private static Base64.Encoder base64Encoder;
+    private static final String HMAC_SHA256 = "HmacSHA256";
+    private static final Charset PROPERTY_CHARSET = StandardCharsets.UTF_8;
+
+    public StandardSensitiveValueEncoder(final NiFiProperties properties) {
+        this(properties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY),
+                SecureHasherFactory.getSecureHasher(properties.getProperty(NiFiProperties.SENSITIVE_PROPS_ALGORITHM)));
+    }
+
+    // We use the sensitive properties key and a SecureHasher impl to derive a secret key for the getEncoded() method
+    private StandardSensitiveValueEncoder(final String sensitivePropertiesKey, final SecureHasher hasher) {
+        Objects.requireNonNull(sensitivePropertiesKey, "Sensitive Properties Key is required");
+        Objects.requireNonNull(hasher, "SecureHasher is required");
+        byte[] hashedSensitivePropertyKey = hasher.hashRaw(sensitivePropertiesKey.getBytes(PROPERTY_CHARSET));
+        secretKeySpec = new SecretKeySpec(hashedSensitivePropertyKey, HMAC_SHA256);
+        base64Encoder = Base64.getEncoder();
+    }
+
+    /**
+     * Creates a securely-derived, deterministic representation of the provided decrypted NiFi property value
+     * for logging/comparison purposes. A SecureHasher implementation is used to derive a secret key from the sensitive which is
+     * then used to generate an HMAC using HMAC+SHA256.
+     *
+     * @param plaintextSensitiveValue A decrypted, sensitive property value
+     *
+     * @return a deterministic, securely hashed representation of the value which will be consistent across nodes. Safe to print in a log.
+     */
+    @Override
+    public String getEncoded(final String plaintextSensitiveValue) {
+        try {
+            Mac mac = Mac.getInstance(HMAC_SHA256);
+            mac.init(secretKeySpec);
+            byte[] hashedBytes = mac.doFinal(plaintextSensitiveValue.getBytes(PROPERTY_CHARSET));
+            return "[MASKED] (" + base64Encoder.encodeToString(hashedBytes) + ")";
+        } catch (NoSuchAlgorithmException | InvalidKeyException e) {
+            logger.error("Encountered an error making the sensitive value loggable: {}", e.getLocalizedMessage());
+            return "[Unable to mask value]";
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 4920bdd..d32d431 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -131,6 +131,8 @@ import org.apache.nifi.diagnostics.StorageUsage;
 import org.apache.nifi.diagnostics.SystemDiagnostics;
 import org.apache.nifi.diagnostics.SystemDiagnosticsFactory;
 import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.encrypt.SensitiveValueEncoder;
+import org.apache.nifi.encrypt.StandardSensitiveValueEncoder;
 import org.apache.nifi.engine.FlowEngine;
 import org.apache.nifi.events.BulletinFactory;
 import org.apache.nifi.events.EventReporter;
@@ -323,6 +325,11 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
      */
     private final PropertyEncryptor encryptor;
 
+    /**
+     * The sensitive value string encoder (hasher)
+     */
+    private final SensitiveValueEncoder sensitiveValueEncoder;
+
     private final ScheduledExecutorService clusterTaskExecutor = new FlowEngine(3, "Clustering Tasks", true);
     private final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager();
 
@@ -475,6 +482,8 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
             throw new IllegalStateException("Flow controller TLS configuration is invalid", e);
         }
 
+        this.sensitiveValueEncoder = new StandardSensitiveValueEncoder(nifiProperties);
+
         timerDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxTimerDrivenThreads.get(), "Timer-Driven Process"));
         eventDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxEventDrivenThreads.get(), "Event-Driven Process"));
 
@@ -1161,6 +1170,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         return encryptor;
     }
 
+    public SensitiveValueEncoder getSensitiveValueEncoder() {
+        return sensitiveValueEncoder;
+    }
+
     /**
      * @return the ExtensionManager used for instantiating Processors,
      * Prioritizers, etc.
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowFingerprintCheck.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowFingerprintCheck.java
index f75ed21..079786d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowFingerprintCheck.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/inheritance/FlowFingerprintCheck.java
@@ -19,6 +19,7 @@ package org.apache.nifi.controller.inheritance;
 import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.encrypt.SensitiveValueEncoder;
 import org.apache.nifi.fingerprint.FingerprintFactory;
 import org.apache.nifi.nar.ExtensionManager;
 import org.slf4j.Logger;
@@ -38,8 +39,9 @@ public class FlowFingerprintCheck implements FlowInheritabilityCheck {
 
         final PropertyEncryptor encryptor = flowController.getEncryptor();
         final ExtensionManager extensionManager = flowController.getExtensionManager();
+        final SensitiveValueEncoder sensitiveValueEncoder = flowController.getSensitiveValueEncoder();
 
-        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager);
+        final FingerprintFactory fingerprintFactory = new FingerprintFactory(encryptor, extensionManager, sensitiveValueEncoder);
         final String existingFlowFingerprintBeforeHash = fingerprintFactory.createFingerprint(existingFlowBytes, flowController);
         if (existingFlowFingerprintBeforeHash.trim().isEmpty()) {
             return null;  // no existing flow, so equivalent to proposed flow
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index bd54c70..345a247 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -16,26 +16,6 @@
  */
 package org.apache.nifi.fingerprint;
 
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.security.InvalidKeyException;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.Base64;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.stream.Stream;
-import javax.crypto.Mac;
-import javax.crypto.spec.SecretKeySpec;
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.ConfigurableComponent;
@@ -44,14 +24,12 @@ import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.serialization.FlowEncodingVersion;
 import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
 import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.encrypt.SensitiveValueEncoder;
 import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.properties.NiFiPropertiesLoader;
-import org.apache.nifi.security.util.crypto.Argon2SecureHasher;
 import org.apache.nifi.security.xml.XmlUtils;
 import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.DomUtils;
 import org.apache.nifi.util.LoggingXmlParserErrorHandler;
-import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.api.dto.BundleDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.ReportingTaskDTO;
@@ -63,6 +41,21 @@ import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.stream.Stream;
+
 /**
  * <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
  *
@@ -92,14 +85,16 @@ public class FingerprintFactory {
     private final PropertyEncryptor encryptor;
     private final DocumentBuilder flowConfigDocBuilder;
     private final ExtensionManager extensionManager;
+    private final SensitiveValueEncoder sensitiveValueEncoder;
 
     private byte[] sensitivePropertyKeyBytes;
 
     private static final Logger logger = LoggerFactory.getLogger(FingerprintFactory.class);
 
-    public FingerprintFactory(final PropertyEncryptor encryptor, final ExtensionManager extensionManager) {
+    public FingerprintFactory(final PropertyEncryptor encryptor, final ExtensionManager extensionManager, final SensitiveValueEncoder sensitiveValueEncoder) {
         this.encryptor = encryptor;
         this.extensionManager = extensionManager;
+        this.sensitiveValueEncoder = sensitiveValueEncoder;
 
         final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
         final Schema schema;
@@ -116,10 +111,11 @@ public class FingerprintFactory {
         }
     }
 
-    public FingerprintFactory(final PropertyEncryptor encryptor, final DocumentBuilder docBuilder, final ExtensionManager extensionManager) {
+    public FingerprintFactory(final PropertyEncryptor encryptor, final DocumentBuilder docBuilder, final ExtensionManager extensionManager, final SensitiveValueEncoder sensitiveValueEncoder) {
         this.encryptor = encryptor;
         this.flowConfigDocBuilder = docBuilder;
         this.extensionManager = extensionManager;
+        this.sensitiveValueEncoder = sensitiveValueEncoder;
     }
 
     /**
@@ -549,34 +545,8 @@ public class FingerprintFactory {
      * @return a deterministic string value which represents this input but is safe to print in a log
      */
     private String getLoggableRepresentationOfSensitiveValue(String encryptedPropertyValue) {
-        initializeSensitivePropertyKeyBytes();
-        final String plaintextValue = decrypt(encryptedPropertyValue);
-        try {
-            Mac mac = Mac.getInstance("HmacSHA256");
-            mac.init(new SecretKeySpec(sensitivePropertyKeyBytes, "HmacSHA256"));
-            byte[] hashedBytes = mac.doFinal(plaintextValue.getBytes(StandardCharsets.UTF_8));
-            return "[MASKED] (" + Base64.getEncoder().encodeToString(hashedBytes) + ")";
-        } catch (NoSuchAlgorithmException | InvalidKeyException e) {
-            // This should not occur on any system which runs NiFi as HmacSHA256 is provided by BouncyCastle and SunJCE
-            logger.error("Encountered an error making the sensitive value loggable: {}", e.getLocalizedMessage());
-            return "[Unable to mask value]";
-        }
-    }
-
-    private void initializeSensitivePropertyKeyBytes() {
-        if (sensitivePropertyKeyBytes == null || sensitivePropertyKeyBytes.length == 0) {
-            Argon2SecureHasher a2sh = new Argon2SecureHasher();
-
-            // Derive the reusable HMAC key from the nifi.sensitive.props.key to ensure deterministic output across nodes
-            try {
-                String npsk = NiFiPropertiesLoader.loadDefaultWithKeyFromBootstrap().getProperty(NiFiProperties.SENSITIVE_PROPS_KEY);
-
-                // The output will be 32B (256b)
-                sensitivePropertyKeyBytes = a2sh.hashRaw(npsk.getBytes(StandardCharsets.UTF_8));
-            } catch (IOException e) {
-                logger.error("Encountered an error loading NiFi properties while fingerprinting flow: ", e);
-            }
-        }
+        final String plaintextValue = encryptor.decrypt(encryptedPropertyValue);
+        return sensitiveValueEncoder.getEncoded(plaintextValue);
     }
 
     private StringBuilder addPortFingerprint(final StringBuilder builder, final Element portElem) throws FingerprintException {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy
index 2d0ad38..2bbc1f8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy
@@ -17,6 +17,7 @@
 package org.apache.nifi.fingerprint
 
 import org.apache.nifi.encrypt.PropertyEncryptor
+import org.apache.nifi.encrypt.SensitiveValueEncoder
 import org.apache.nifi.nar.ExtensionManager
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager
 import org.apache.nifi.util.NiFiProperties
@@ -40,6 +41,8 @@ class FingerprintFactoryGroovyTest extends GroovyTestCase {
     private static PropertyEncryptor mockEncryptor = [
             encrypt: { String plaintext -> plaintext.reverse() },
             decrypt: { String cipherText -> cipherText.reverse() }] as PropertyEncryptor
+    private static SensitiveValueEncoder mockSensitiveValueEncoder = [
+            getEncoded: { String plaintext -> "[MASKED] (${plaintext.sha256()})".toString() }] as SensitiveValueEncoder
     private static ExtensionManager extensionManager = new StandardExtensionDiscoveringManager()
 
     private static String originalPropertiesPath = System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH)
@@ -84,7 +87,7 @@ class FingerprintFactoryGroovyTest extends GroovyTestCase {
         logger.info("Read initial flow: ${initialFlowXML[0..<100]}...")
 
         // Create the FingerprintFactory with collaborators
-        FingerprintFactory fingerprintFactory = new FingerprintFactory(mockEncryptor, extensionManager)
+        FingerprintFactory fingerprintFactory = new FingerprintFactory(mockEncryptor, extensionManager, mockSensitiveValueEncoder)
 
         // Act
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
index 14ea913..f68cd89 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/fingerprint/FingerprintFactoryTest.java
@@ -16,29 +16,6 @@
  */
 package org.apache.nifi.fingerprint;
 
-import static org.apache.nifi.controller.serialization.ScheduledStateLookup.IDENTITY_LOOKUP;
-import static org.apache.nifi.fingerprint.FingerprintFactory.FLOW_CONFIG_XSD;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.File;
-import java.io.IOException;
-import java.lang.reflect.Method;
-import java.nio.charset.StandardCharsets;
-import java.util.Base64;
-import java.util.Collections;
-import java.util.Optional;
-import javax.crypto.Mac;
-import javax.crypto.spec.SecretKeySpec;
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.connectable.Position;
@@ -48,10 +25,11 @@ import org.apache.nifi.controller.serialization.FlowSerializer;
 import org.apache.nifi.controller.serialization.ScheduledStateLookup;
 import org.apache.nifi.controller.serialization.StandardFlowSerializer;
 import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.encrypt.SensitiveValueEncoder;
+import org.apache.nifi.encrypt.StandardSensitiveValueEncoder;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
-import org.apache.nifi.properties.NiFiPropertiesLoader;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 import org.apache.nifi.security.util.crypto.Argon2SecureHasher;
@@ -66,6 +44,30 @@ import org.xml.sax.ErrorHandler;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXParseException;
 
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.Optional;
+
+import static org.apache.nifi.controller.serialization.ScheduledStateLookup.IDENTITY_LOOKUP;
+import static org.apache.nifi.fingerprint.FingerprintFactory.FLOW_CONFIG_XSD;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 /**
  */
 public class FingerprintFactoryTest {
@@ -73,15 +75,19 @@ public class FingerprintFactoryTest {
     private PropertyEncryptor encryptor;
     private ExtensionManager extensionManager;
     private FingerprintFactory fingerprinter;
+    private SensitiveValueEncoder sensitiveValueEncoder;
+    private NiFiProperties niFiProperties;
 
     private static final String ORIGINAL_NIFI_PROPS_PATH = System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH);
     private static final String TEST_NIFI_PROPS_PATH = "src/test/resources/conf/nifi.properties";
 
     @Before
     public void setup() {
+        niFiProperties = NiFiProperties.createBasicNiFiProperties(TEST_NIFI_PROPS_PATH);
         encryptor = createEncryptor();
+        sensitiveValueEncoder = new StandardSensitiveValueEncoder(niFiProperties);
         extensionManager = new StandardExtensionDiscoveringManager();
-        fingerprinter = new FingerprintFactory(encryptor, extensionManager);
+        fingerprinter = new FingerprintFactory(encryptor, extensionManager, sensitiveValueEncoder);
     }
 
     @AfterClass
@@ -154,7 +160,7 @@ public class FingerprintFactoryTest {
 
     @Test
     public void testSchemaValidation() throws IOException {
-        FingerprintFactory fp = new FingerprintFactory(null, getValidatingDocumentBuilder(), extensionManager);
+        FingerprintFactory fp = new FingerprintFactory(null, getValidatingDocumentBuilder(), extensionManager, null);
         final String fingerprint = fp.createFingerprint(getResourceBytes("/nifi/fingerprint/validating-flow.xml"), null);
     }
 
@@ -283,8 +289,7 @@ public class FingerprintFactoryTest {
         Argon2SecureHasher a2sh = new Argon2SecureHasher();
 
         // The nifi.properties file needs to be present
-        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TEST_NIFI_PROPS_PATH);
-        String npsk = NiFiPropertiesLoader.loadDefaultWithKeyFromBootstrap().getProperty(NiFiProperties.SENSITIVE_PROPS_KEY);
+        String npsk = niFiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY);
 
         // The output will be 32B (256b)
         byte[] sensitivePropertyKeyBytes = a2sh.hashRaw(npsk.getBytes(StandardCharsets.UTF_8));
@@ -388,7 +393,16 @@ public class FingerprintFactoryTest {
 
             @Override
             public String decrypt(String encryptedProperty) {
-                return encryptedProperty;
+                return encryptedProperty.substring(4, encryptedProperty.length() - 1);
+            }
+        };
+    }
+
+    private SensitiveValueEncoder createSensitiveValueEncoder() {
+        return new SensitiveValueEncoder() {
+            @Override
+            public String getEncoded(String sensitivePropertyValue) {
+                return String.format("[MASKED] %s", sensitivePropertyValue);
             }
         };
     }