You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2020/01/31 14:04:49 UTC

[nifi] 01/02: NIFI-7079 Added SecureHasher interface and Argon2SecureHasher implementation. Added unit tests. Fixed JwtService charset references. Changed FingerprintFactory to mask sensitive values via Argon2 hashing. Resolved RAT checks.

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

joewitt pushed a commit to branch support/nifi-1.11.x
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0b71034278bf366ce14de2f2c811a1c93acd644e
Author: Andy LoPresto <al...@apache.org>
AuthorDate: Thu Jan 30 20:51:32 2020 -0800

    NIFI-7079 Added SecureHasher interface and Argon2SecureHasher implementation.
    Added unit tests.
    Fixed JwtService charset references.
    Changed FingerprintFactory to mask sensitive values via Argon2 hashing.
    Resolved RAT checks.
---
 nifi-commons/nifi-security-utils/pom.xml           |   9 +-
 .../security/util/crypto/Argon2SecureHasher.java   | 218 +++++++++++++++++++
 .../nifi/security/util/crypto/SecureHasher.java    |  56 +++++
 .../util/crypto/Argon2SecureHasherTest.groovy      | 242 +++++++++++++++++++++
 .../src/test/resources/logback-test.xml            |  11 +-
 .../nifi/fingerprint/FingerprintFactory.java       |  75 +++++--
 .../FingerprintFactoryGroovyTest.groovy            |  88 ++++++++
 .../nifi/fingerprint/FingerprintFactoryTest.java   |  50 +++--
 .../src/test/resources/logback-test.xml            |   1 +
 .../test/resources/nifi/fingerprint/initial.xml    | 152 +++++++++++++
 .../test/resources/nifi/fingerprint/modified.xml   | 152 +++++++++++++
 .../nifi/web/security/jwt/JwtServiceTest.java      |  45 ++--
 12 files changed, 1016 insertions(+), 83 deletions(-)

diff --git a/nifi-commons/nifi-security-utils/pom.xml b/nifi-commons/nifi-security-utils/pom.xml
index 475d4a0..2fd4c37 100644
--- a/nifi-commons/nifi-security-utils/pom.xml
+++ b/nifi-commons/nifi-security-utils/pom.xml
@@ -37,9 +37,14 @@
             <artifactId>nifi-utils</artifactId>
             <version>1.11.1-SNAPSHOT</version>
         </dependency>
+        <!--<dependency>-->
+            <!--<groupId>org.slf4j</groupId>-->
+            <!--<artifactId>slf4j-api</artifactId>-->
+        <!--</dependency>-->
         <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-api</artifactId>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+            <scope>provided</scope>
         </dependency>
         <dependency>
             <groupId>org.apache.commons</groupId>
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
new file mode 100644
index 0000000..c152759
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/Argon2SecureHasher.java
@@ -0,0 +1,218 @@
+/*
+ * 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 java.nio.charset.StandardCharsets;
+import java.security.SecureRandom;
+import java.util.concurrent.TimeUnit;
+import org.bouncycastle.crypto.generators.Argon2BytesGenerator;
+import org.bouncycastle.crypto.params.Argon2Parameters;
+import org.bouncycastle.util.encoders.Base64;
+import org.bouncycastle.util.encoders.Hex;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provides an implementation of {@code Argon2} for secure password hashing. This class is
+ * roughly based on Spring Security's implementation but does not include the full module
+ * in this utility module. This implementation uses {@code Argon2id} which provides a
+ * balance of protection against side-channel and memory attacks.
+ * <p>
+ * One <strong>critical</strong> difference is that this implementation uses a
+ * <strong>static universal</strong> salt unless instructed otherwise, which provides
+ * strict determinism across nodes in a cluster. The purpose for this is to allow for
+ * blind equality comparison of sensitive values hashed on different nodes (with
+ * potentially different {@code nifi.sensitive.props.key} values) during flow inheritance
+ * (see {@code FingerprintFactory}).
+ */
+public class Argon2SecureHasher implements SecureHasher {
+    private static final Logger logger = LoggerFactory.getLogger(Argon2SecureHasher.class);
+
+    private static final int DEFAULT_SALT_LENGTH = 16;
+    private static final int DEFAULT_HASH_LENGTH = 32;
+    private static final int DEFAULT_PARALLELISM = 1;
+    private static final int DEFAULT_MEMORY = 1 << 12;
+    private static final int DEFAULT_ITERATIONS = 3;
+
+    private final int hashLength;
+    private final int memory;
+    private final int parallelism;
+    private final int iterations;
+    private final int saltLength;
+
+    private final boolean usingStaticSalt;
+
+    // A 16 byte salt (nonce) is recommended for password hashing
+    private static final byte[] staticSalt = "NiFi Static Salt".getBytes(StandardCharsets.UTF_8);
+
+    /**
+     * Instantiates an Argon2 secure hasher using the default cost parameters
+     * ({@code hashLength = }{@link #DEFAULT_HASH_LENGTH},
+     * {@code memory = }{@link #DEFAULT_MEMORY},
+     * {@code parallelism = }{@link #DEFAULT_PARALLELISM},
+     * {@code iterations = }{@link #DEFAULT_ITERATIONS}). A static salt is also used.
+     */
+    public Argon2SecureHasher() {
+        this(DEFAULT_HASH_LENGTH, DEFAULT_MEMORY, DEFAULT_PARALLELISM, DEFAULT_ITERATIONS, 0);
+    }
+
+    /**
+     * Instantiates an Argon2 secure hasher using the provided cost parameters. A unique
+     * {@link #DEFAULT_SALT_LENGTH} byte salt will be generated on every hash request.
+     *
+     * @param hashLength  the output length in bytes ({@code 4 to 2^32 -1})
+     * @param memory      the integer number of KB used ({@code 8p to 2^32 - 1})
+     * @param parallelism degree of parallelism ({@code 1 to 2^24 - 1})
+     * @param iterations  number of iterations ({@code 1 to 2^32 - 1})
+     */
+    public Argon2SecureHasher(int hashLength, int memory, int parallelism, int iterations) {
+        this(hashLength, memory, parallelism, iterations, 0);
+    }
+
+    /**
+     * Instantiates an Argon2 secure hasher using the provided cost parameters. A unique
+     * salt of the specified length will be generated on every hash request.
+     *
+     * @param hashLength  the output length in bytes ({@code 4 to 2^32 -1})
+     * @param memory      the integer number of KB used ({@code 8p to 2^32 - 1})
+     * @param parallelism degree of parallelism ({@code 1 to 2^24 - 1})
+     * @param iterations  number of iterations ({@code 1 to 2^32 - 1})
+     * @param saltLength  the salt length in bytes {@code 8 to 2^32 - 1})
+     */
+    public Argon2SecureHasher(int hashLength, int memory, int parallelism, int iterations, int saltLength) {
+        // TODO: Implement boundary checking
+        this.hashLength = hashLength;
+        this.memory = memory;
+        this.parallelism = parallelism;
+        this.iterations = iterations;
+
+        this.saltLength = saltLength;
+        if (saltLength > 0) {
+            this.usingStaticSalt = false;
+        } else {
+            this.usingStaticSalt = true;
+            logger.debug("Configured to use static salt");
+        }
+    }
+
+    /**
+     * Returns {@code true} if this instance is configured to use a static salt.
+     *
+     * @return true if all hashes will be generated using a static salt
+     */
+    public boolean isUsingStaticSalt() {
+        return usingStaticSalt;
+    }
+
+    /**
+     * Returns a salt to use. If using a static salt (see {@link #isUsingStaticSalt()}),
+     * this return value will be identical across every invocation. If using a dynamic salt,
+     * it will be {@link #saltLength} bytes of a securely-generated random value.
+     *
+     * @return the salt value
+     */
+    byte[] getSalt() {
+        if (isUsingStaticSalt()) {
+            return staticSalt;
+        } else {
+            SecureRandom sr = new SecureRandom();
+            byte[] salt = new byte[saltLength];
+            sr.nextBytes(salt);
+            return salt;
+        }
+    }
+
+    /**
+     * Returns a String representation of {@code CHF(input)} in hex-encoded format.
+     *
+     * @param input the input
+     * @return the hex-encoded hash
+     */
+    @Override
+    public String hashHex(String input) {
+        if (input == null) {
+            logger.warn("Attempting to generate an Argon2 hash of null input; using empty input");
+            input = "";
+        }
+
+        return Hex.toHexString(hash(input.getBytes(StandardCharsets.UTF_8)));
+    }
+
+    /**
+     * Returns a String representation of {@code CHF(input)} in Base 64-encoded format.
+     *
+     * @param input the input
+     * @return the Base 64-encoded hash
+     */
+    @Override
+    public String hashBase64(String input) {
+        if (input == null) {
+            logger.warn("Attempting to generate an Argon2 hash of null input; using empty input");
+            input = "";
+        }
+
+        return Base64.toBase64String(hash(input.getBytes(StandardCharsets.UTF_8)));
+    }
+
+    /**
+     * Returns a byte[] representation of {@code CHF(input)}.
+     *
+     * @param input the input
+     * @return the hash
+     */
+    @Override
+    public byte[] hashRaw(byte[] input) {
+        return hash(input);
+    }
+
+    /**
+     * Internal method to hash the raw bytes.
+     *
+     * @param input the raw bytes to hash (can be length 0)
+     * @return the generated hash
+     */
+    private byte[] hash(byte[] input) {
+        byte[] salt = getSalt();
+        byte[] hash = new byte[hashLength];
+        logger.debug("Creating {} byte Argon2 hash with salt [{}]", hashLength, Hex.toHexString(salt));
+
+        final long startNanos = System.nanoTime();
+
+        Argon2Parameters params = new Argon2Parameters.Builder(Argon2Parameters.ARGON2_id)
+                .withSalt(salt)
+                .withParallelism(parallelism)
+                .withMemoryAsKB(memory)
+                .withIterations(iterations)
+                .build();
+        Argon2BytesGenerator generator = new Argon2BytesGenerator();
+        generator.init(params);
+
+        final long initNanos = System.nanoTime();
+
+        generator.generateBytes(input, hash);
+
+        final long generateNanos = System.nanoTime();
+
+        final long initDurationMicros = TimeUnit.NANOSECONDS.toMicros(initNanos - startNanos);
+        final long generateDurationMicros = TimeUnit.NANOSECONDS.toMicros(generateNanos - initNanos);
+        final long totalDurationMillis = TimeUnit.MICROSECONDS.toMillis(initDurationMicros + generateDurationMicros);
+
+        logger.debug("Generated Argon2 hash in {} ms (init: {} µs, generate: {} µs)", totalDurationMillis, initDurationMicros, generateDurationMicros);
+
+        return hash;
+    }
+}
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasher.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasher.java
new file mode 100644
index 0000000..be0328f
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/SecureHasher.java
@@ -0,0 +1,56 @@
+/*
+ * 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;
+
+/**
+ * An interface which specifies that implementations should provide a
+ * cryptographic hash function (CHF) which accepts input and returns a
+ * deterministic, (mathematically-difficult) irreversible value.
+ *
+ * While SHA-256, SHA-512, and Blake2 are CHF implementations, this interface is intended to
+ * be used by password protection or key derivation functions (KDF) like
+ * {@link PBKDF2CipherProvider}, {@link BcryptCipherProvider}, {@link ScryptCipherProvider},
+ * or {@link Argon2SecureHasher}. These classes implement iterative processes which make use
+ * of cryptographic primitives to return an irreversible value which can either securely
+ * store a password representation or be used as an encryption key derived from a password.
+ */
+public interface SecureHasher {
+
+    /**
+     * Returns a String representation of {@code CHF(input)} in hex-encoded format.
+     *
+     * @param input the input
+     * @return the hex-encoded hash
+     */
+    String hashHex(String input);
+
+    /**
+     * Returns a String representation of {@code CHF(input)} in Base 64-encoded format.
+     *
+     * @param input the input
+     * @return the Base 64-encoded hash
+     */
+    String hashBase64(String input);
+
+    /**
+     * Returns a byte[] representation of {@code CHF(input)}.
+     *
+     * @param input the input
+     * @return the hash
+     */
+    byte[] hashRaw(byte[] input);
+}
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.groovy
new file mode 100644
index 0000000..e952a70
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/crypto/Argon2SecureHasherTest.groovy
@@ -0,0 +1,242 @@
+/*
+ * 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.kerby.util.Hex
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Ignore
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class Argon2SecureHasherTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(Argon2SecureHasherTest.class)
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() throws Exception {
+    }
+
+    @After
+    void tearDown() throws Exception {
+    }
+
+    private static byte[] decodeHex(String hex) {
+        Hex.decode(hex?.replaceAll("[^0-9a-fA-F]", ""))
+    }
+
+    @Ignore("Cannot override static salt")
+    @Test
+    void testShouldMatchReferenceVectors() {
+        // Arrange
+        int hashLength = 32
+        int memory = 32
+        int parallelism = 4
+        int iterations = 3
+        logger.info("Generating Argon2 hash for hash length: ${hashLength} B, mem: ${memory} KiB, parallelism: ${parallelism}, iterations: ${iterations}")
+
+        Argon2SecureHasher a2sh = new Argon2SecureHasher(hashLength, memory, parallelism, iterations)
+        // Override the static salt for the published test vector
+//        a2sh.staticSalt = [0x02] * 16
+
+        // Act
+        byte[] hash = a2sh.hashRaw([0x01] * 32 as byte[])
+        logger.info("Generated hash: ${Hex.encode(hash)}")
+
+        // Assert
+        assert hash == decodeHex("0d 64 0d f5 8d 78 76 6c 08 c0 37 a3 4a 8b 53 c9 d0 " +
+                "1e f0 45 2d 75 b6 5e b5 25 20 e9 6b 01 e6 59")
+
+        // Clean up
+//        Argon2SecureHasher.staticSalt = "NiFi Static Salt".bytes
+    }
+
+    @Test
+    void testShouldBeDeterministicWithStaticSalt() {
+        // Arrange
+        int hashLength = 32
+        int memory = 8
+        int parallelism = 4
+        int iterations = 4
+        logger.info("Generating Argon2 hash for hash length: ${hashLength} B, mem: ${memory} KiB, parallelism: ${parallelism}, iterations: ${iterations}")
+
+        int testIterations = 10
+        byte[] inputBytes = "This is a sensitive value".bytes
+
+        final String EXPECTED_HASH_HEX = "a73a471f51b2900901a00b81e770b9c1dfc595602bb7aec64cd27754a4174919"
+
+        Argon2SecureHasher a2sh = new Argon2SecureHasher(hashLength, memory, parallelism, iterations)
+
+        def results = []
+
+        // Act
+        testIterations.times { int i ->
+            byte[] hash = a2sh.hashRaw(inputBytes)
+            String hashHex = Hex.encode(hash)
+            logger.info("Generated hash: ${hashHex}")
+            results << hashHex
+        }
+
+        // Assert
+        assert results.every { it == EXPECTED_HASH_HEX }
+    }
+
+    @Test
+    void testShouldBeDifferentWithRandomSalt() {
+        // Arrange
+        int hashLength = 32
+        int memory = 8
+        int parallelism = 4
+        int iterations = 4
+        logger.info("Generating Argon2 hash for hash length: ${hashLength} B, mem: ${memory} KiB, parallelism: ${parallelism}, iterations: ${iterations}")
+
+        int testIterations = 10
+        byte[] inputBytes = "This is a sensitive value".bytes
+
+        final String EXPECTED_HASH_HEX = "a73a471f51b2900901a00b81e770b9c1dfc595602bb7aec64cd27754a4174919"
+
+        Argon2SecureHasher a2sh = new Argon2SecureHasher(hashLength, memory, parallelism, iterations, 16)
+
+        def results = []
+
+        // Act
+        testIterations.times { int i ->
+            byte[] hash = a2sh.hashRaw(inputBytes)
+            String hashHex = Hex.encode(hash)
+            logger.info("Generated hash: ${hashHex}")
+            results << hashHex
+        }
+
+        // Assert
+        assert results.unique().size() == results.size()
+        assert results.every { it != EXPECTED_HASH_HEX }
+    }
+
+    @Test
+    void testShouldFormatHex() {
+        // Arrange
+        String input = "This is a sensitive value"
+
+        final String EXPECTED_HASH_HEX = "0c2920c52f28e0a2c77d006ec6138c8dc59580881468b85541cf886abdebcf18"
+
+        Argon2SecureHasher a2sh = new Argon2SecureHasher()
+
+        // Act
+        String hashHex = a2sh.hashHex(input)
+        logger.info("Generated hash: ${hashHex}")
+
+        // Assert
+        assert hashHex == EXPECTED_HASH_HEX
+    }
+
+    @Test
+    void testShouldFormatBase64() {
+        // Arrange
+        String input = "This is a sensitive value"
+
+        final String EXPECTED_HASH_B64 = "DCkgxS8o4KLHfQBuxhOMjcWVgIgUaLhVQc+Iar3rzxg="
+
+        Argon2SecureHasher a2sh = new Argon2SecureHasher()
+
+        // Act
+        String hashB64 = a2sh.hashBase64(input)
+        logger.info("Generated hash: ${hashB64}")
+
+        // Assert
+        assert hashB64 == EXPECTED_HASH_B64
+    }
+
+    @Test
+    void testShouldHandleNullInput() {
+        // Arrange
+        List<String> inputs = [null, ""]
+
+        final String EXPECTED_HASH_HEX = "8e5625a66b94ed9d31c1496d7f9ff49249cf05d6753b50ba0e2bf2a1108973dd"
+        final String EXPECTED_HASH_B64 = "jlYlpmuU7Z0xwUltf5/0kknPBdZ1O1C6DivyoRCJc90="
+
+        Argon2SecureHasher a2sh = new Argon2SecureHasher()
+
+        def hexResults = []
+        def b64Results = []
+
+        // Act
+        inputs.each { String input ->
+            String hashHex = a2sh.hashHex(input)
+            logger.info("Generated hash: ${hashHex}")
+            hexResults << hashHex
+
+            String hashB64 = a2sh.hashBase64(input)
+            logger.info("Generated hash: ${hashB64}")
+            b64Results << hashB64
+        }
+
+        // Assert
+        assert hexResults.every { it == EXPECTED_HASH_HEX }
+        assert b64Results.every { it == EXPECTED_HASH_B64 }
+    }
+
+    /**
+     * This test can have the minimum time threshold updated to determine if the performance
+     * is still sufficient compared to the existing threat model.
+     */
+    @Test
+    void testDefaultCostParamsShouldBeSufficient() {
+        // Arrange
+        int testIterations = 10
+        byte[] inputBytes = "This is a sensitive value".bytes
+
+        Argon2SecureHasher a2sh = new Argon2SecureHasher()
+
+        def results = []
+        def resultDurations = []
+
+        // Act
+        testIterations.times { int i ->
+            long startNanos = System.nanoTime()
+            byte[] hash = a2sh.hashRaw(inputBytes)
+            long endNanos = System.nanoTime()
+            long durationNanos = endNanos - startNanos
+
+            String hashHex = Hex.encode(hash)
+            logger.info("Generated hash: ${hashHex} in ${durationNanos} ns")
+
+            results << hashHex
+            resultDurations << durationNanos
+        }
+
+        // Assert
+        final long MIN_DURATION_NANOS = 5_000_000 // 5 ms
+        assert resultDurations.min() > MIN_DURATION_NANOS
+        assert resultDurations.sum() / testIterations > MIN_DURATION_NANOS
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml b/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
similarity index 74%
copy from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
copy to nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
index 076ba32..d8e72bd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
+++ b/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
@@ -30,16 +30,9 @@
 
 
     <logger name="org.apache.nifi" level="INFO"/>
-    <logger name="org.apache.nifi.controller.tasks" level="DEBUG" />"
-    <logger name="org.apache.nifi.controller.service" level="DEBUG"/>
-    <logger name="org.apache.nifi.encrypt" level="DEBUG"/>
-    <logger name="org.apache.nifi.controller.repository.crypto" level="DEBUG"/>
-    <logger name="org.apache.nifi.security.repository" level="DEBUG"/>
-    <logger name="org.apache.nifi.controller.service.mock" level="ERROR"/>
+    <logger name="org.apache.nifi.security.util.crypto" level="DEBUG"/>
 
-    <logger name="StandardProcessSession.claims" level="INFO" />
-
-    <root level="INFO">
+    <root level="DEBUG">
         <appender-ref ref="CONSOLE"/>
     </root>
 
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 35d081b..15e1635 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,6 +16,23 @@
  */
 package org.apache.nifi.fingerprint;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.security.NoSuchAlgorithmException;
+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;
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+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;
@@ -25,6 +42,8 @@ import org.apache.nifi.controller.serialization.FlowEncodingVersion;
 import org.apache.nifi.controller.serialization.FlowFromDOMFactory;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.security.util.crypto.Argon2SecureHasher;
+import org.apache.nifi.security.util.crypto.SecureHasher;
 import org.apache.nifi.util.BundleUtils;
 import org.apache.nifi.util.DomUtils;
 import org.apache.nifi.util.LoggingXmlParserErrorHandler;
@@ -39,24 +58,6 @@ 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.parsers.DocumentBuilderFactory;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.security.NoSuchAlgorithmException;
-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.
  *
@@ -342,7 +343,7 @@ public class FingerprintFactory {
         // append value
         if (isEncrypted(value)) {
             // propValue is non null, no need to use getValue
-            builder.append(decrypt(value));
+            builder.append(getLoggableRepresentationOfSensitiveValue(value));
         } else {
             builder.append(getValue(value, NO_VALUE));
         }
@@ -532,8 +533,8 @@ public class FingerprintFactory {
 
         // append value
         if (isEncrypted(propValue)) {
-            // propValue is non null, no need to use getValue
-            builder.append(decrypt(propValue));
+            // Get a secure, deterministic, loggable representation of this value
+            builder.append(getLoggableRepresentationOfSensitiveValue(propValue));
         } else {
             builder.append(getValue(propValue, NO_VALUE));
         }
@@ -541,6 +542,32 @@ public class FingerprintFactory {
         return builder;
     }
 
+    /**
+     * Returns a securely-derived, deterministic value from the provided encrypted property
+     * value. This is because the flow fingerprint is displayed in the log if NiFi has
+     * trouble inheriting a flow, so the sensitive value should not be disclosed through the
+     * log. However, the equality or difference of the sensitive value can influence in the
+     * inheritability of the flow, 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 encryptedPropertyValue the encrypted property value
+     * @return a deterministic string value which represents this input but is safe to print in a log
+     */
+    private String getLoggableRepresentationOfSensitiveValue(String encryptedPropertyValue) {
+        // TODO: Implement Scrypt or Argon2 secure hash of decrypted value
+
+        // 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 {@link StringEncryptor} with secure hashing capability and inject?
+        String hexEncodedHash = secureHasher.hashHex(decrypt(encryptedPropertyValue));
+        return hexEncodedHash;
+    }
+
     private StringBuilder addPortFingerprint(final StringBuilder builder, final Element portElem) throws FingerprintException {
         // id
         appendFirstValue(builder, DomUtils.getChildNodesByTagName(portElem, "id"));
@@ -593,7 +620,7 @@ public class FingerprintFactory {
                 "transportProtocol", "proxyHost", "proxyPort", "proxyUser", "proxyPassword"}) {
             final String value = getFirstValue(DomUtils.getChildNodesByTagName(remoteProcessGroupElem, tagName));
             if (isEncrypted(value)) {
-                builder.append(decrypt(value));
+                builder.append(getLoggableRepresentationOfSensitiveValue(value));
             } else {
                 builder.append(value);
             }
@@ -824,7 +851,7 @@ public class FingerprintFactory {
                 final String e1PropName = getFirstValue(DomUtils.getChildNodesByTagName(e1, "name"));
                 String e1PropValue = getFirstValue(DomUtils.getChildNodesByTagName(e1, "value"));
                 if (isEncrypted(e1PropValue)) {
-                    e1PropValue = decrypt(e1PropValue);
+                    e1PropValue = getLoggableRepresentationOfSensitiveValue(e1PropValue);
                 }
                 final String e1CombinedValue = e1PropName + e1PropValue;
 
@@ -832,7 +859,7 @@ public class FingerprintFactory {
                 final String e2PropName = getFirstValue(DomUtils.getChildNodesByTagName(e2, "name"));
                 String e2PropValue = getFirstValue(DomUtils.getChildNodesByTagName(e2, "value"));
                 if (isEncrypted(e2PropValue)) {
-                    e2PropValue = decrypt(e2PropValue);
+                    e2PropValue = getLoggableRepresentationOfSensitiveValue(e2PropValue);
                 }
                 final String e2CombinedValue = e2PropName + e2PropValue;
 
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
new file mode 100644
index 0000000..969cad8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/fingerprint/FingerprintFactoryGroovyTest.groovy
@@ -0,0 +1,88 @@
+/*
+ * 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.fingerprint
+
+
+import org.apache.nifi.encrypt.StringEncryptor
+import org.apache.nifi.nar.ExtensionManager
+import org.apache.nifi.nar.StandardExtensionDiscoveringManager
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class FingerprintFactoryGroovyTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(FingerprintFactoryGroovyTest.class)
+
+    private static StringEncryptor mockEncryptor = [
+            encrypt: { String plaintext -> plaintext.reverse() },
+            decrypt: { String cipherText -> cipherText.reverse() }] as StringEncryptor
+    private static ExtensionManager extensionManager = new StandardExtensionDiscoveringManager()
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() throws Exception {
+
+    }
+
+    @After
+    void tearDown() throws Exception {
+
+    }
+
+    /**
+     * The flow fingerprint should not disclose sensitive property values.
+     */
+    @Test
+    void testCreateFingerprintShouldNotDiscloseSensitivePropertyValues() {
+        // Arrange
+
+        // Create flow
+        String initialFlowXML = new File("src/test/resources/nifi/fingerprint/initial.xml").text
+        logger.info("Read initial flow: ${initialFlowXML[0..<100]}...")
+
+        // Create the FingerprintFactory with collaborators
+        FingerprintFactory fingerprintFactory = new FingerprintFactory(mockEncryptor, extensionManager)
+
+        // Act
+
+        // Create the fingerprint from the flow
+        String fingerprint = fingerprintFactory.createFingerprint(initialFlowXML.bytes)
+        logger.info("Generated flow fingerprint: ${fingerprint}")
+
+        // Assert
+
+        // Assert the fingerprint does not contain the password
+        assert !(fingerprint =~ "originalPlaintextPassword")
+    }
+}
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 35bc2e3..5baaf6d 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,6 +16,26 @@
  */
 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.util.Collections;
+import java.util.Optional;
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+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;
@@ -30,6 +50,7 @@ import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
+import org.apache.nifi.security.util.crypto.Argon2SecureHasher;
 import org.junit.Before;
 import org.junit.Test;
 import org.w3c.dom.Document;
@@ -38,27 +59,6 @@ import org.xml.sax.ErrorHandler;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXParseException;
 
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-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.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 {
@@ -248,6 +248,7 @@ public class FingerprintFactoryTest {
 
     @Test
     public void testRemoteProcessGroupFingerprintWithProxy() throws Exception {
+        final String proxyPassword = "proxy-pass";
 
         // Fill out every configuration.
         final RemoteProcessGroup component = mock(RemoteProcessGroup.class);
@@ -263,10 +264,11 @@ public class FingerprintFactoryTest {
         when(component.getProxyHost()).thenReturn("proxy-host");
         when(component.getProxyPort()).thenReturn(3128);
         when(component.getProxyUser()).thenReturn("proxy-user");
-        when(component.getProxyPassword()).thenReturn("proxy-pass");
+        when(component.getProxyPassword()).thenReturn(proxyPassword);
         when(component.getVersionedComponentId()).thenReturn(Optional.empty());
 
         // Assert fingerprints with expected one.
+        final String hashedProxyPassword = new Argon2SecureHasher().hashHex(proxyPassword);
         final String expected = "id" +
                 "NO_VALUE" +
                 "http://node1:8080/nifi, http://node2:8080/nifi" +
@@ -277,11 +279,11 @@ public class FingerprintFactoryTest {
                 "proxy-host" +
                 "3128" +
                 "proxy-user" +
-                "proxy-pass";
+                hashedProxyPassword;
 
         final Element rootElement = serializeElement(encryptor, RemoteProcessGroup.class, component, "addRemoteProcessGroup", IDENTITY_LOOKUP);
         final Element componentElement = (Element) rootElement.getElementsByTagName("remoteProcessGroup").item(0);
-        assertEquals(expected.toString(), fingerprint("addRemoteProcessGroupFingerprint", Element.class, componentElement));
+        assertEquals(expected, fingerprint("addRemoteProcessGroupFingerprint", Element.class, componentElement));
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
index 076ba32..99d1743 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml
@@ -33,6 +33,7 @@
     <logger name="org.apache.nifi.controller.tasks" level="DEBUG" />"
     <logger name="org.apache.nifi.controller.service" level="DEBUG"/>
     <logger name="org.apache.nifi.encrypt" level="DEBUG"/>
+    <logger name="org.apache.nifi.security.util.crypto" level="DEBUG"/>
     <logger name="org.apache.nifi.controller.repository.crypto" level="DEBUG"/>
     <logger name="org.apache.nifi.security.repository" level="DEBUG"/>
     <logger name="org.apache.nifi.controller.service.mock" level="ERROR"/>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi/fingerprint/initial.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi/fingerprint/initial.xml
new file mode 100644
index 0000000..ef37e51
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi/fingerprint/initial.xml
@@ -0,0 +1,152 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  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.
+-->
+<flowController encoding-version="1.4">
+    <maxTimerDrivenThreadCount>10</maxTimerDrivenThreadCount>
+    <maxEventDrivenThreadCount>1</maxEventDrivenThreadCount>
+    <registries/>
+    <parameterContexts/>
+    <rootGroup>
+        <id>f31f71c7-016f-1000-52b1-4f4e21375b31</id>
+        <name>NiFi Flow</name>
+        <position x="0.0" y="0.0"/>
+        <comment/>
+        <processor>
+            <id>f3214b1c-016f-1000-0000-00001b1546be</id>
+            <name>GetFTP</name>
+            <position x="363.0" y="111.01388549804688"/>
+            <styles/>
+            <comment/>
+            <class>org.apache.nifi.processors.standard.GetFTP</class>
+            <bundle>
+                <group>org.apache.nifi</group>
+                <artifact>nifi-standard-nar</artifact>
+                <version>1.11.0</version>
+            </bundle>
+            <maxConcurrentTasks>1</maxConcurrentTasks>
+            <schedulingPeriod>0 sec</schedulingPeriod>
+            <penalizationPeriod>30 sec</penalizationPeriod>
+            <yieldPeriod>1 sec</yieldPeriod>
+            <bulletinLevel>WARN</bulletinLevel>
+            <lossTolerant>false</lossTolerant>
+            <scheduledState>STOPPED</scheduledState>
+            <schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
+            <executionNode>ALL</executionNode>
+            <runDurationNanos>0</runDurationNanos>
+            <property>
+                <name>Hostname</name>
+                <value>abc</value>
+            </property>
+            <property>
+                <name>Port</name>
+                <value>21</value>
+            </property>
+            <property>
+                <name>Username</name>
+                <value>abc</value>
+            </property>
+            <property>
+                <name>Password</name>
+                <value>enc{drowssaPtxetnialPlanigiro}</value>
+            </property>
+            <property>
+                <name>Connection Mode</name>
+                <value>Passive</value>
+            </property>
+            <property>
+                <name>Transfer Mode</name>
+                <value>Binary</value>
+            </property>
+            <property>
+                <name>Remote Path</name>
+            </property>
+            <property>
+                <name>File Filter Regex</name>
+            </property>
+            <property>
+                <name>Path Filter Regex</name>
+            </property>
+            <property>
+                <name>Polling Interval</name>
+                <value>60 sec</value>
+            </property>
+            <property>
+                <name>Search Recursively</name>
+                <value>false</value>
+            </property>
+            <property>
+                <name>follow-symlink</name>
+                <value>false</value>
+            </property>
+            <property>
+                <name>Ignore Dotted Files</name>
+                <value>true</value>
+            </property>
+            <property>
+                <name>Delete Original</name>
+                <value>true</value>
+            </property>
+            <property>
+                <name>Connection Timeout</name>
+                <value>30 sec</value>
+            </property>
+            <property>
+                <name>Data Timeout</name>
+                <value>30 sec</value>
+            </property>
+            <property>
+                <name>Max Selects</name>
+                <value>100</value>
+            </property>
+            <property>
+                <name>Remote Poll Batch Size</name>
+                <value>5000</value>
+            </property>
+            <property>
+                <name>Use Natural Ordering</name>
+                <value>false</value>
+            </property>
+            <property>
+                <name>proxy-configuration-service</name>
+            </property>
+            <property>
+                <name>Proxy Type</name>
+                <value>DIRECT</value>
+            </property>
+            <property>
+                <name>Proxy Host</name>
+            </property>
+            <property>
+                <name>Proxy Port</name>
+            </property>
+            <property>
+                <name>Http Proxy Username</name>
+            </property>
+            <property>
+                <name>Http Proxy Password</name>
+            </property>
+            <property>
+                <name>Internal Buffer Size</name>
+                <value>16KB</value>
+            </property>
+            <property>
+                <name>ftp-use-utf8</name>
+                <value>false</value>
+            </property>
+        </processor>
+    </rootGroup>
+    <controllerServices/>
+    <reportingTasks/>
+</flowController>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi/fingerprint/modified.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi/fingerprint/modified.xml
new file mode 100644
index 0000000..922d0f9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/nifi/fingerprint/modified.xml
@@ -0,0 +1,152 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+  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.
+-->
+<flowController encoding-version="1.4">
+    <maxTimerDrivenThreadCount>10</maxTimerDrivenThreadCount>
+    <maxEventDrivenThreadCount>1</maxEventDrivenThreadCount>
+    <registries/>
+    <parameterContexts/>
+    <rootGroup>
+        <id>f31f71c7-016f-1000-52b1-4f4e21375b31</id>
+        <name>NiFi Flow</name>
+        <position x="0.0" y="0.0"/>
+        <comment/>
+        <processor>
+            <id>f3214b1c-016f-1000-0000-00001b1546be</id>
+            <name>GetFTP</name>
+            <position x="363.0" y="111.01388549804688"/>
+            <styles/>
+            <comment/>
+            <class>org.apache.nifi.processors.standard.GetFTP</class>
+            <bundle>
+                <group>org.apache.nifi</group>
+                <artifact>nifi-standard-nar</artifact>
+                <version>1.11.0</version>
+            </bundle>
+            <maxConcurrentTasks>1</maxConcurrentTasks>
+            <schedulingPeriod>0 sec</schedulingPeriod>
+            <penalizationPeriod>30 sec</penalizationPeriod>
+            <yieldPeriod>1 sec</yieldPeriod>
+            <bulletinLevel>WARN</bulletinLevel>
+            <lossTolerant>false</lossTolerant>
+            <scheduledState>STOPPED</scheduledState>
+            <schedulingStrategy>TIMER_DRIVEN</schedulingStrategy>
+            <executionNode>ALL</executionNode>
+            <runDurationNanos>0</runDurationNanos>
+            <property>
+                <name>Hostname</name>
+                <value>abc</value>
+            </property>
+            <property>
+                <name>Port</name>
+                <value>21</value>
+            </property>
+            <property>
+                <name>Username</name>
+                <value>abc</value>
+            </property>
+            <property>
+                <name>Password</name>
+                <value>enc{drowssaPtxetnialPdeifidom}</value>
+            </property>
+            <property>
+                <name>Connection Mode</name>
+                <value>Passive</value>
+            </property>
+            <property>
+                <name>Transfer Mode</name>
+                <value>Binary</value>
+            </property>
+            <property>
+                <name>Remote Path</name>
+            </property>
+            <property>
+                <name>File Filter Regex</name>
+            </property>
+            <property>
+                <name>Path Filter Regex</name>
+            </property>
+            <property>
+                <name>Polling Interval</name>
+                <value>60 sec</value>
+            </property>
+            <property>
+                <name>Search Recursively</name>
+                <value>false</value>
+            </property>
+            <property>
+                <name>follow-symlink</name>
+                <value>false</value>
+            </property>
+            <property>
+                <name>Ignore Dotted Files</name>
+                <value>true</value>
+            </property>
+            <property>
+                <name>Delete Original</name>
+                <value>true</value>
+            </property>
+            <property>
+                <name>Connection Timeout</name>
+                <value>30 sec</value>
+            </property>
+            <property>
+                <name>Data Timeout</name>
+                <value>30 sec</value>
+            </property>
+            <property>
+                <name>Max Selects</name>
+                <value>100</value>
+            </property>
+            <property>
+                <name>Remote Poll Batch Size</name>
+                <value>5000</value>
+            </property>
+            <property>
+                <name>Use Natural Ordering</name>
+                <value>false</value>
+            </property>
+            <property>
+                <name>proxy-configuration-service</name>
+            </property>
+            <property>
+                <name>Proxy Type</name>
+                <value>DIRECT</value>
+            </property>
+            <property>
+                <name>Proxy Host</name>
+            </property>
+            <property>
+                <name>Proxy Port</name>
+            </property>
+            <property>
+                <name>Http Proxy Username</name>
+            </property>
+            <property>
+                <name>Http Proxy Password</name>
+            </property>
+            <property>
+                <name>Internal Buffer Size</name>
+                <value>16KB</value>
+            </property>
+            <property>
+                <name>ftp-use-utf8</name>
+                <value>false</value>
+            </property>
+        </processor>
+    </rootGroup>
+    <controllerServices/>
+    <reportingTasks/>
+</flowController>
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
index 17754d0..0727ccf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/jwt/JwtServiceTest.java
@@ -16,8 +16,22 @@
  */
 package org.apache.nifi.web.security.jwt;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 import io.jsonwebtoken.JwtException;
-import org.apache.commons.codec.CharEncoding;
+import java.nio.charset.StandardCharsets;
+import java.security.InvalidKeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.nifi.admin.service.AdministrationException;
 import org.apache.nifi.admin.service.KeyService;
@@ -40,22 +54,6 @@ import org.springframework.security.core.Authentication;
 import org.springframework.security.core.context.SecurityContext;
 import org.springframework.security.core.context.SecurityContextHolder;
 
-import javax.crypto.Mac;
-import javax.crypto.spec.SecretKeySpec;
-import java.io.UnsupportedEncodingException;
-import java.security.InvalidKeyException;
-import java.security.NoSuchAlgorithmException;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.ArgumentMatchers.anyInt;
-import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class JwtServiceTest {
 
     private static final Logger logger = LoggerFactory.getLogger(JwtServiceTest.class);
@@ -155,8 +153,8 @@ public class JwtServiceTest {
         try {
             logger.info("Generating token for " + rawHeader + " + " + rawPayload);
 
-            String base64Header = Base64.encodeBase64URLSafeString(rawHeader.getBytes(CharEncoding.UTF_8));
-            String base64Payload = Base64.encodeBase64URLSafeString(rawPayload.getBytes(CharEncoding.UTF_8));
+            String base64Header = Base64.encodeBase64URLSafeString(rawHeader.getBytes(StandardCharsets.UTF_8));
+            String base64Payload = Base64.encodeBase64URLSafeString(rawPayload.getBytes(StandardCharsets.UTF_8));
             // TODO: Support valid/invalid manipulation
 
             final String body = base64Header + TOKEN_DELIMITER + base64Payload;
@@ -164,7 +162,7 @@ public class JwtServiceTest {
             String signature = generateHMAC(hmacSecret, body);
 
             return body + TOKEN_DELIMITER + signature;
-        } catch (NoSuchAlgorithmException | InvalidKeyException | UnsupportedEncodingException e) {
+        } catch (NoSuchAlgorithmException | InvalidKeyException e) {
             final String errorMessage = "Could not generate the token";
             logger.error(errorMessage, e);
             fail(errorMessage);
@@ -173,11 +171,11 @@ public class JwtServiceTest {
     }
 
     private static String generateHMAC(String hmacSecret, String body) throws NoSuchAlgorithmException,
-            UnsupportedEncodingException, InvalidKeyException {
+            InvalidKeyException {
         Mac hmacSHA256 = Mac.getInstance("HmacSHA256");
-        SecretKeySpec secret_key = new SecretKeySpec(hmacSecret.getBytes("UTF-8"), "HmacSHA256");
+        SecretKeySpec secret_key = new SecretKeySpec(hmacSecret.getBytes(StandardCharsets.UTF_8), "HmacSHA256");
         hmacSHA256.init(secret_key);
-        return Base64.encodeBase64URLSafeString(hmacSHA256.doFinal(body.getBytes("UTF-8")));
+        return Base64.encodeBase64URLSafeString(hmacSHA256.doFinal(body.getBytes(StandardCharsets.UTF_8)));
     }
 
     @Before
@@ -481,7 +479,6 @@ public class JwtServiceTest {
 
     @Test
     public void testShouldLogOutUser() throws Exception {
-
         // Arrange
         expectedException.expect(JwtException.class);
         expectedException.expectMessage("Unable to validate the access token.");