You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ym...@apache.org on 2018/04/13 23:26:40 UTC

[1/2] nifi git commit: NIFI-4942 [WIP] Added skeleton for secure hash handling in encrypt-config toolkit. Added test resource for Python scrypt implementation/verifier. Added unit tests.

Repository: nifi
Updated Branches:
  refs/heads/master 82ac81553 -> 6d06defa6


http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
index 6b056d7..12ed84f 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/properties/ConfigEncryptionToolTest.groovy
@@ -16,15 +16,22 @@
  */
 package org.apache.nifi.properties
 
+import groovy.json.JsonBuilder
+import groovy.json.JsonSlurper
+import org.apache.commons.cli.CommandLine
+import org.apache.commons.cli.CommandLineParser
+import org.apache.commons.cli.DefaultParser
 import org.apache.commons.lang3.SystemUtils
 import org.apache.log4j.AppenderSkeleton
 import org.apache.log4j.spi.LoggingEvent
 import org.apache.nifi.encrypt.StringEncryptor
+import org.apache.nifi.security.util.crypto.scrypt.Scrypt
 import org.apache.nifi.toolkit.tls.commandLine.CommandLineParseException
 import org.apache.nifi.util.NiFiProperties
 import org.apache.nifi.util.console.TextDevice
 import org.apache.nifi.util.console.TextDevices
 import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.bouncycastle.util.encoders.Hex
 import org.junit.After
 import org.junit.AfterClass
 import org.junit.Assume
@@ -65,6 +72,17 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
     private static final String KEY_HEX_256 = KEY_HEX_128 * 2
     public static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
     private static final String PASSWORD = "thisIsABadPassword"
+
+    private static final String STATIC_SALT = "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUV"
+    private static final String SCRYPT_SALT_PATTERN = /\$\w{2}\$\w{5,}\$[\w\/\=\+]+/
+
+    // Hash of "password" with 00 * 16 salt
+    private static
+    final String HASHED_PASSWORD = "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM"
+    // Hash of [key derived from "password"] with 00 * 16 salt
+    private static
+    final String HASHED_KEY_HEX = "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$pJOGA9sPL+pRzynnwt6G2FfVTyLQdbKSbk6W8IKId8E"
+
     // From ConfigEncryptionTool.deriveKeyFromPassword("thisIsABadPassword")
     private static
     final String PASSWORD_KEY_HEX_256 = "2C576A9585DB862F5ECBEE5B4FFFCCA14B18D8365968D7081651006507AD2BDE"
@@ -107,6 +125,8 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
 
     @Before
     void setUp() throws Exception {
+        // Manually override the constant path to allow for easy cleanup
+        ConfigEncryptionTool.secureHashPath = "target/tmp/secure_hash.key"
     }
 
     @After
@@ -406,6 +426,81 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
     }
 
     @Test
+    void testParseShouldFailIfMigrationPasswordAndHashedPasswordBothProvided() {
+        // Arrange
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+
+        // Act
+        def msg = shouldFail {
+            tool.parse("-m -n nifi.properties -w oldPassword -z oldPasswordHashed".split(" ") as String[])
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert msg =~ "If the '-w'/'--oldPassword' or '-e'/'--oldKey' arguments are present, '-z'/'--secureHashPassword' and '-y'/'--secureHashKey' cannot be used"
+    }
+
+    @Test
+    void testParseShouldFailIfMigrationPasswordAndHashedKeyBothProvided() {
+        // Arrange
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+
+        // Act
+        def msg = shouldFail {
+            tool.parse("-m -n nifi.properties -w oldPassword -y oldKeyHashed".split(" ") as String[])
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert msg =~ "If the '-w'/'--oldPassword' or '-e'/'--oldKey' arguments are present, '-z'/'--secureHashPassword' and '-y'/'--secureHashKey' cannot be used"
+    }
+
+    @Test
+    void testParseShouldFailIfMigrationKeyAndHashedPasswordBothProvided() {
+        // Arrange
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+
+        // Act
+        def msg = shouldFail {
+            tool.parse("-m -n nifi.properties -e oldKey -z oldPasswordHashed".split(" ") as String[])
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert msg =~ "If the '-w'/'--oldPassword' or '-e'/'--oldKey' arguments are present, '-z'/'--secureHashPassword' and '-y'/'--secureHashKey' cannot be used"
+    }
+
+    @Test
+    void testParseShouldFailIfMigrationKeyAndHashedKeyBothProvided() {
+        // Arrange
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+
+        // Act
+        def msg = shouldFail {
+            tool.parse("-m -n nifi.properties -e oldKey -y oldKeyHashed".split(" ") as String[])
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert msg =~ "If the '-w'/'--oldPassword' or '-e'/'--oldKey' arguments are present, '-z'/'--secureHashPassword' and '-y'/'--secureHashKey' cannot be used"
+    }
+
+    @Test
+    void testParseShouldFailIfHashedPasswordAndHashedKeyBothProvided() {
+        // Arrange
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+
+        // Act
+        def msg = shouldFail {
+            tool.parse("-m -n nifi.properties -z oldPasswordHashed -y oldKeyHashed".split(" ") as String[])
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert msg =~ "Only one of '-z'/'--secureHashPassword' and '-y'/'--secureHashKey' can be used together"
+    }
+
+    @Test
     void testParseShouldFailIfPropertiesAndProvidersMissing() {
         // Arrange
         ConfigEncryptionTool tool = new ConfigEncryptionTool()
@@ -1441,8 +1536,7 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert msg == "The nifi.properties file at ${workingFile.path} must be writable by the user running this tool".toString()
 
         workingFile.deleteOnExit()
-        setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE])
-        tmpDir.deleteOnExit()
+        setupTmpDir()
     }
 
     @Ignore("Setting the Windows file permissions fails in the test harness, so the test does not throw the expected exception")
@@ -1915,6 +2009,621 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         // Assertions in common method above
     }
 
+    /**
+     * Helper method to execute key migration test for varying combinations of new key/password with securely hashed key/password.
+     *
+     * @param scenario a human-readable description of the test scenario
+     * @param scenarioArgs a list of the arguments specific to this scenario to be passed to the tool
+     * @param oldHashedPassword the secure hash of the original password
+     * @param newPassword the new password
+     * @param oldHashedKeyHex the original key hex (if present, original hashed password is ignored)
+     * @param newKeyHex the new key hex (if present, new password is ignored; if not, this is derived)
+     */
+    private void performSecureHashKeyMigration(String scenario, List scenarioArgs, String oldHashedPassword = HASHED_PASSWORD, String newPassword = PASSWORD.reverse(), String oldHashedKeyHex = "", String newKeyHex = "", int desiredExitCode = 0) {
+        // Arrange
+        exit.expectSystemExitWithStatus(desiredExitCode)
+
+        // Initial set up
+        File tmpDir = new File("target/tmp/")
+        tmpDir.mkdirs()
+        setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
+
+        String bootstrapPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/bootstrap_with_master_key_password.conf" :
+                "src/test/resources/bootstrap_with_master_key_password_128.conf"
+        File originalKeyFile = new File(bootstrapPath)
+        File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
+        bootstrapFile.delete()
+
+        Files.copy(originalKeyFile.toPath(), bootstrapFile.toPath())
+        final List<String> originalBootstrapLines = bootstrapFile.readLines()
+        String originalKeyLine = originalBootstrapLines.find {
+            it.startsWith(ConfigEncryptionTool.BOOTSTRAP_KEY_PREFIX)
+        }
+
+        // Copy the hashed credentials file
+        String secureHashedPasswordPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/secure_hash.key" :
+                "src/test/resources/secure_hash_128.key"
+        File originalSecureHashedPasswordFile = new File(secureHashedPasswordPath)
+        File secureHashedFile = new File("target/tmp/tmp_secure_hash.key")
+        secureHashedFile.delete()
+        Files.copy(originalSecureHashedPasswordFile.toPath(), secureHashedFile.toPath())
+
+        // Perform necessary key derivations
+        if (!newKeyHex) {
+            newKeyHex = ConfigEncryptionTool.deriveKeyFromPassword(newPassword)
+            logger.info("Migration key derived from password [${newPassword}]: \t${newKeyHex}")
+        } else {
+            logger.info("Migration key provided directly: \t${newKeyHex}")
+        }
+
+        // Extract old key hex from bootstrap.conf
+        String oldKeyHex = originalKeyLine.split("=", 2).last()
+        logger.info("Extracted old key hex from bootstrap.conf: ${oldKeyHex}")
+
+        String inputPropertiesPath = isUnlimitedStrengthCryptoAvailable() ?
+                "src/test/resources/nifi_with_sensitive_properties_protected_aes_password.properties" :
+                "src/test/resources/nifi_with_sensitive_properties_protected_aes_password_128.properties"
+        File inputPropertiesFile = new File(inputPropertiesPath)
+        File outputPropertiesFile = new File("target/tmp/tmp_nifi.properties")
+        outputPropertiesFile.delete()
+
+        // Log original sensitive properties (encrypted with first key)
+        NiFiProperties inputProperties = NiFiPropertiesLoader.withKey(oldKeyHex).load(inputPropertiesFile)
+        logger.info("Loaded ${inputProperties.size()} properties from input file")
+        ProtectedNiFiProperties protectedInputProperties = new ProtectedNiFiProperties(inputProperties)
+        def originalSensitiveValues = protectedInputProperties.getSensitivePropertyKeys().collectEntries { String key -> [(key): protectedInputProperties.getProperty(key)] }
+        logger.info("Original sensitive values: ${originalSensitiveValues}")
+
+        final String EXPECTED_NEW_KEY_LINE = ConfigEncryptionTool.BOOTSTRAP_KEY_PREFIX + newKeyHex
+
+        // Act
+        String[] args = ["-n", inputPropertiesFile.path,
+                         "-b", bootstrapFile.path,
+                         "-o", outputPropertiesFile.path,
+                         "-m",
+                         "-v"]
+
+        List<String> localArgs = args + scenarioArgs
+        logger.info("Running [${scenario}] with args: ${localArgs}")
+
+        // If an error is expected, check that the nifi.properties file doesn't exist (i.e. nothing happened)
+        Assertion assertion
+        if (desiredExitCode != 0) {
+            assertion = new Assertion() {
+                void checkAssertion() {
+                    assert !outputPropertiesFile.exists()
+                    logger.expected("No output nifi.properties found")
+
+                    // Check that the key was NOT persisted to the bootstrap.conf
+                    final List<String> updatedBootstrapLines = bootstrapFile.readLines()
+                    String updatedKeyLine = updatedBootstrapLines.find {
+                        it.startsWith(ConfigEncryptionTool.BOOTSTRAP_KEY_PREFIX)
+                    }
+                    logger.info("'Updated' key line: ${updatedKeyLine}")
+
+                    assert updatedKeyLine == originalKeyLine
+                    assert originalBootstrapLines.size() == updatedBootstrapLines.size()
+
+                    // Check that the secure hash was NOT persisted to the secure_hash.key
+                    final List<String> updatedSecureHashLines = secureHashedFile.readLines()
+                    String updatedSecureHashKeyLine = updatedSecureHashLines.find { it.startsWith("secureHashKey=") }
+                    logger.info("'Updated' secure hash lines: \n${updatedSecureHashLines.join("\n")}")
+
+                    int expectedSecureHashLineCount = 1
+
+                    List<String> originalSecureHashLines = originalSecureHashedPasswordFile.readLines()
+
+                    // Only evaluate the secure hash password line if the raw password was provided (otherwise, can't store hash)
+                    if (newPassword) {
+                        String updatedSecureHashPasswordLine = updatedSecureHashLines.find {
+                            it.startsWith("secureHashPassword=")
+                        }
+                        expectedSecureHashLineCount = 2
+
+                        logger.info("Asserting \n${updatedSecureHashPasswordLine} == \n${originalSecureHashLines.last()}")
+                        assert updatedSecureHashPasswordLine == originalSecureHashLines.last()
+                    }
+
+                    logger.info("Asserting \n${updatedSecureHashKeyLine} == \n${originalSecureHashLines.first()}")
+                    assert updatedSecureHashKeyLine == originalSecureHashLines.first()
+                    assert updatedSecureHashLines.size() == expectedSecureHashLineCount
+
+                    // Clean up
+                    outputPropertiesFile.deleteOnExit()
+                    bootstrapFile.deleteOnExit()
+                    tmpDir.deleteOnExit()
+                    secureHashedFile.deleteOnExit()
+                }
+            }
+        } else {
+            assertion = new Assertion() {
+                void checkAssertion() {
+                    assert outputPropertiesFile.exists()
+                    final List<String> updatedPropertiesLines = outputPropertiesFile.readLines()
+                    logger.info("Updated nifi.properties:")
+                    logger.info("\n" * 2 + updatedPropertiesLines.join("\n"))
+
+                    // Check that the output values for sensitive properties are not the same as the original (i.e. it was re-encrypted)
+                    NiFiProperties updatedProperties = new NiFiPropertiesLoader().readProtectedPropertiesFromDisk(outputPropertiesFile)
+                    assert updatedProperties.size() >= inputProperties.size()
+                    originalSensitiveValues.every { String key, String originalValue ->
+                        assert updatedProperties.getProperty(key) != originalValue
+                    }
+
+                    // Check that the new NiFiProperties instance matches the output file (values still encrypted)
+                    updatedProperties.getPropertyKeys().every { String key ->
+                        assert updatedPropertiesLines.contains("${key}=${updatedProperties.getProperty(key)}".toString())
+                    }
+
+                    // Check that the key was persisted to the bootstrap.conf
+                    final List<String> updatedBootstrapLines = bootstrapFile.readLines()
+                    String updatedKeyLine = updatedBootstrapLines.find {
+                        it.startsWith(ConfigEncryptionTool.BOOTSTRAP_KEY_PREFIX)
+                    }
+                    logger.info("Updated key line: ${updatedKeyLine}")
+
+                    assert updatedKeyLine == EXPECTED_NEW_KEY_LINE
+                    assert originalBootstrapLines.size() == updatedBootstrapLines.size()
+
+                    // Check that the secure hash was persisted to the secure_hash.key
+                    final List<String> updatedSecureHashLines = secureHashedFile.readLines()
+                    String updatedSecureHashKeyLine = updatedSecureHashLines.find { it.startsWith("secureHashKey=") }
+                    logger.info("Updated secure hash lines: \n${updatedSecureHashLines.join("\n")}")
+
+                    int expectedSecureHashLineCount = 1
+
+                    // Extract the salt(s) so the credentials can be hashed with the same salt
+                    final String keySalt = updatedSecureHashKeyLine.find(SCRYPT_SALT_PATTERN)
+                    logger.info("Extracted key salt:      ${keySalt}")
+
+                    final String EXPECTED_NEW_SECURE_HASH_KEY_LINE = "secureHashKey=${ConfigEncryptionTool.secureHashKey(newKeyHex, keySalt)}"
+
+                    // Only evaluate the secure hash password line if the raw password was provided (otherwise, can't store hash)
+                    if (newPassword) {
+                        String updatedSecureHashPasswordLine = updatedSecureHashLines.find {
+                            it.startsWith("secureHashPassword=")
+                        }
+                        final String passwordSalt = updatedSecureHashPasswordLine.find(SCRYPT_SALT_PATTERN)
+                        logger.info("Extracted password salt: ${passwordSalt}")
+                        final String EXPECTED_NEW_SECURE_HASH_PASSWORD_LINE = "secureHashPassword=${ConfigEncryptionTool.secureHashPassword(newPassword, passwordSalt)}"
+                        expectedSecureHashLineCount = 2
+
+                        logger.info("Asserting \n${updatedSecureHashPasswordLine} == \n${EXPECTED_NEW_SECURE_HASH_PASSWORD_LINE}")
+                        assert updatedSecureHashPasswordLine == EXPECTED_NEW_SECURE_HASH_PASSWORD_LINE
+                    }
+
+                    logger.info("Asserting \n${updatedSecureHashKeyLine} == \n${EXPECTED_NEW_SECURE_HASH_KEY_LINE}")
+                    assert updatedSecureHashKeyLine == EXPECTED_NEW_SECURE_HASH_KEY_LINE
+                    assert updatedSecureHashLines.size() == expectedSecureHashLineCount
+
+                    // Clean up
+                    outputPropertiesFile.deleteOnExit()
+                    bootstrapFile.deleteOnExit()
+                    tmpDir.deleteOnExit()
+                    secureHashedFile.deleteOnExit()
+                }
+            }
+        }
+
+        exit.checkAssertionAfterwards(assertion)
+
+        logger.info("Migrating key (${scenario}) with ${localArgs.join(" ")}")
+
+        // Override the "final" secure hash file path
+        ConfigEncryptionTool.secureHashPath = secureHashedFile.path
+        ConfigEncryptionTool.main(localArgs as String[])
+
+        // Assert
+
+        // Assertions defined above
+    }
+
+    /**
+     * Ideally all of the combination tests would be a single test with iterative argument lists, but due to the System.exit(), it can only be captured once per test.
+     */
+    @Test
+    void testShouldMigrateFromHashedPasswordToPassword() {
+        // Arrange
+        String scenario = "hashed password to password"
+        def args = ["-z", HASHED_PASSWORD, "-p", PASSWORD.reverse()]
+
+        // Act
+        performSecureHashKeyMigration(scenario, args, HASHED_PASSWORD, PASSWORD.reverse())
+
+        // Assert
+
+        // Assertions in common method above
+    }
+
+    @Test
+    void testShouldMigrateFromHashedPasswordToKey() {
+        // Arrange
+        String scenario = "hashed password to key"
+        def args = ["-z", HASHED_PASSWORD, "-k", KEY_HEX]
+
+        // Act
+        performSecureHashKeyMigration(scenario, args, HASHED_PASSWORD, "", "", KEY_HEX)
+
+        // Assert
+
+        // Assertions in common method above
+    }
+
+    @Test
+    void testShouldMigrateFromHashedKeyToPassword() {
+        // Arrange
+        String scenario = "hashed key to password"
+        def args = ["-y", HASHED_KEY_HEX, "-p", PASSWORD.reverse()]
+
+        // Act
+        performSecureHashKeyMigration(scenario, args, "", PASSWORD.reverse(), HASHED_KEY_HEX, "")
+
+        // Assert
+
+        // Assertions in common method above
+    }
+
+    @Test
+    void testShouldMigrateFromHashedKeyToKey() {
+        // Arrange
+        String scenario = "hashed key to key"
+        def args = ["-y", HASHED_KEY_HEX, "-k", KEY_HEX]
+
+        // Act
+        performSecureHashKeyMigration(scenario, args, "", "", HASHED_KEY_HEX, KEY_HEX)
+
+        // Assert
+
+        // Assertions in common method above
+    }
+
+    @Test
+    void testShouldFailToMigrateFromIncorrectHashedPasswordToPassword() {
+        // Arrange
+        String scenario = "(incorrect) hashed password to password"
+        final String INCORRECT_HASHED_PASSWORD = "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$thisIsDefinitelyNotTheCorrectPasswordHashxx"
+        def args = ["-z", INCORRECT_HASHED_PASSWORD, "-p", PASSWORD.reverse()]
+
+        // Act
+        performSecureHashKeyMigration(scenario, args, HASHED_PASSWORD, PASSWORD.reverse(), "", "", 4)
+
+        // Assert
+
+        // Assertions in common method above
+    }
+
+    @Test
+    void testShouldDeriveSecureHashOfPassword() {
+        // Arrange
+        def testPasswords = ["password", "thisIsABadPassword", "bWZerzZo6fw9ZrDz*YfM6CVj2Ktx(YJd"]
+
+        // All zero, 22 (16B) Base64 static, 40 (32B) Base64 randomly-generated
+        def salts = [
+                Hex.decode("00" * 16),
+                Base64.decoder.decode("ABCDEFGHIJKLMNOPQRSTUV"),
+                Base64.decoder.decode("eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc=")
+        ]
+
+        // These values were generated using CET#secureHashPassword() and verified using src/test/resources/scrypt.py
+        def passwordHashes = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$hxU5g0eH6sRkBqcsiApI8jxvKRT+2QMCenV0GToiMQ8",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$99aTTB39TJo69aZCONQmRdyWOgYsDi+1MI+8D0EgMNM"
+        ]
+        def badPasswordHashes = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$Gk7K9YmlsWbd8FS7e4RKVWnkg9vlsqYnlD593pJ71gg",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$Ri78VZbrp2cCVmGh2a9Nbfdov8LPnFb49MYyzPCaXmE",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$rZIrP2qdIY7LN4CZAMgbCzl3YhXz6WhaNyXJXqFIjaI"
+        ]
+        def randomPasswordHashes = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$GxH68bGykmPDZ6gaPIGOONOT2omlZ7cd0xlcZ9UsY/0",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$KLGZjWlo59sbCbtmTg5b4k0Nu+biWZRRzhPhN7K5kkI",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$6Ql6Efd2ac44ERoV31CL3Q0J3LffNZKN4elyMHux99Y"
+        ]
+
+        def expectedHashes = [
+                (testPasswords[0]): passwordHashes,
+                (testPasswords[1]): badPasswordHashes,
+                (testPasswords[2]): randomPasswordHashes
+        ]
+
+        // Low cost factors for performance
+        int n = 2**4
+        int r = 8
+        int p = 1
+        logger.info("Cost factors for test: N=${n}, R=${r}, P=${p}")
+
+        // Act
+        testPasswords.each { String password ->
+            salts.eachWithIndex { byte[] rawSalt, int i ->
+                logger.info("Hashing '${password}' with salt ${Base64.encoder.encodeToString(rawSalt)}")
+                String formattedSalt = Scrypt.formatSalt(rawSalt, n, r, p)
+                logger.info("Formatted salt: ${formattedSalt}")
+                String generatedHash = ConfigEncryptionTool.secureHashPassword(password, formattedSalt)
+                logger.info("Generated hash: ${generatedHash}")
+
+                // Assert
+                String expectedHash = expectedHashes[(password)][i]
+                logger.info("Comparing to expectedHashes['${password}'][${i}]: ${expectedHash}")
+
+                // Remember to perform constant-time equality check in production code
+                assert generatedHash == expectedHash
+            }
+        }
+    }
+
+    @Test
+    void testShouldDeriveSecureHashOfKey() {
+        // Arrange
+        def testKeys = [
+                "00" * 32,
+                "0123456789ABCDEFFEDCBA98765432100123456789ABCDEFFEDCBA9876543210",
+                "0123456789abcdeffedcba98765432100123456789abcdeffedcba9876543210"
+        ]
+
+        // All zero, 22 (16B) Base64 static, 40 (32B) Base64 randomly-generated
+        def salts = [
+                Hex.decode("00" * 16),
+                Base64.decoder.decode("ABCDEFGHIJKLMNOPQRSTUV"),
+                Base64.decoder.decode("eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc=")
+        ]
+
+        // These values were generated using CET#secureHashKey() and verified using src/test/resources/scrypt.py
+        def zeroHashes = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$pOoIk4K9OPYxusXBFNGtEaoHzIIxlgDOTiVO9OiLJrE",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$kQJ7CeAt5qHK4/r2lMnuBzNyBt1h1WDDkmgXH7N0hRc",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$diExTMVvETmC6gjKx+9ITn1L/0FOYNHeQq2oPLMsFvY"
+        ]
+        def uppercaseHashes = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$K5uQBtbkmq2b2M1H6kX/U7g5QiPgmoLCuJYfpOar8w4",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$TbPrKP7+/xPlc74L15QFG+iDqIysPW/dOFVRaj4Rk/k",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$yGpGz7FyBE3nf8Ed/o84o8Glyd4m091HxdVQEhN55zI"
+        ]
+
+        // Should be identical to uppercase hashes due to case-normalization in method
+        def lowercaseHashes = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$K5uQBtbkmq2b2M1H6kX/U7g5QiPgmoLCuJYfpOar8w4",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$TbPrKP7+/xPlc74L15QFG+iDqIysPW/dOFVRaj4Rk/k",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$yGpGz7FyBE3nf8Ed/o84o8Glyd4m091HxdVQEhN55zI"
+        ]
+
+        def expectedHashes = [
+                (testKeys[0]): zeroHashes,
+                (testKeys[1]): uppercaseHashes,
+                (testKeys[2]): lowercaseHashes
+        ]
+
+        // Low cost factors for performance
+        int n = 2**4
+        int r = 8
+        int p = 1
+        logger.info("Cost factors for test: N=${n}, R=${r}, P=${p}")
+
+        // Act
+        testKeys.each { String key ->
+            salts.eachWithIndex { byte[] rawSalt, int i ->
+                logger.info("Hashing '${key}' with salt ${Base64.encoder.encodeToString(rawSalt)}")
+                String formattedSalt = Scrypt.formatSalt(rawSalt, n, r, p)
+                logger.info("Formatted salt: ${formattedSalt}")
+                String generatedHash = ConfigEncryptionTool.secureHashKey(key, formattedSalt)
+                logger.info("Generated hash: ${generatedHash}")
+
+                // Assert
+                String expectedHash = expectedHashes[(key)][i]
+                logger.info("Comparing to expectedHashes['${key}'][${i}]: ${expectedHash}")
+
+                // Remember to perform constant-time equality check in production code
+                assert generatedHash == expectedHash
+            }
+        }
+    }
+
+    @Test
+    void testShouldVerifySecureHashOfPassword() {
+        // Arrange
+        String password = "password"
+
+        // This is a known existing hash of "password"
+        String existingHashedPassword = "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM"
+        logger.info("Known existing hash: ${existingHashedPassword}")
+
+        // Low cost factors for performance
+        int n = 2**4
+        int r = 8
+        int p = 1
+        logger.info("Cost factors for test: N=${n}, R=${r}, P=${p}")
+
+        byte[] rawSalt = Hex.decode("00" * 16)
+
+        // This is a generated hash of "password"
+        String formattedSalt = Scrypt.formatSalt(rawSalt, n, r, p)
+        logger.info("Formatted salt: ${formattedSalt}")
+        String hashedPassword = ConfigEncryptionTool.secureHashPassword(password, formattedSalt)
+        logger.info("Generated hash: ${hashedPassword}")
+
+        // This is a generated hash of "password" using a different salt
+        byte[] otherRawSalt = Hex.decode("01" * 16)
+        String otherFormattedSalt = Scrypt.formatSalt(otherRawSalt, n, r, p)
+        logger.info("Formatted salt: ${otherFormattedSalt}")
+        String otherHashedPassword = ConfigEncryptionTool.secureHashPassword(password, otherFormattedSalt)
+        logger.info("Generated hash: ${otherHashedPassword}")
+
+        // Act
+        logger.info("Checking \n${existingHashedPassword} against hash(${password}, ${formattedSalt}) -> \n${hashedPassword}")
+        boolean hashIsIdentical = ConfigEncryptionTool.checkHashedValue(existingHashedPassword, hashedPassword)
+        logger.info("Hash values equal: ${hashIsIdentical}")
+
+        logger.info("Checking \n${existingHashedPassword} against hash(${password}, ${otherFormattedSalt}) -> \n${otherHashedPassword}")
+        boolean otherHashIsIdentical = ConfigEncryptionTool.checkHashedValue(existingHashedPassword, otherHashedPassword)
+        logger.info("Hash values equal: ${otherHashIsIdentical}")
+
+        // Assert
+        assert hashIsIdentical
+        assert !otherHashIsIdentical
+    }
+
+    @Test
+    void testCheckHashedValueShouldVerifyScryptFormat() {
+        // Arrange
+        def validHashes = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$hxU5g0eH6sRkBqcsiApI8jxvKRT+2QMCenV0GToiMQ8",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$99aTTB39TJo69aZCONQmRdyWOgYsDi+1MI+8D0EgMNM",
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$Gk7K9YmlsWbd8FS7e4RKVWnkg9vlsqYnlD593pJ71gg",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$Ri78VZbrp2cCVmGh2a9Nbfdov8LPnFb49MYyzPCaXmE",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$rZIrP2qdIY7LN4CZAMgbCzl3YhXz6WhaNyXJXqFIjaI",
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$GxH68bGykmPDZ6gaPIGOONOT2omlZ7cd0xlcZ9UsY/0",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$KLGZjWlo59sbCbtmTg5b4k0Nu+biWZRRzhPhN7K5kkI",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$6Ql6Efd2ac44ERoV31CL3Q0J3LffNZKN4elyMHux99Y"
+        ]
+
+        // Some of these are valid "scrypt" hashes but do not conform to the additional requirements NiFi imposes
+        def invalidHashes = [
+                "\$s1\$40801\$AAAAAAA\$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM",
+                "\$s0\$\$ABCDEFGHIJKLMNOPQRSTUQ\$hxU5g0eH6sRkBqcsiApI8jxvKRT+2QMCenV0GToiMQ8",
+                "\$s0\$40801\$\$99aTTB39TJo69aZCONQmRdyWOgYsDi+1MI+8D0EgMNM",
+                "\$s0\$40801\$!!!!\$Gk7K9YmlsWbd8FS7e4RKVWnkg9vlsqYnlD593pJ71gg",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$xxxx",
+        ]
+
+        // Low cost factors for performance
+        int n = 2**4
+        int r = 8
+        int p = 1
+        logger.info("Cost factors for test: N=${n}, R=${r}, P=${p}")
+
+        // Act
+        logger.info("Checking ${validHashes.size()} valid hashes")
+        validHashes.each { String hash ->
+            logger.info("Verifying hash format: ${hash}")
+            boolean validFormat = ConfigEncryptionTool.verifyHashFormat(hash)
+            logger.info("Valid format: ${validFormat}")
+
+            // Assert
+            assert validFormat
+        }
+
+        logger.info("Checking ${invalidHashes.size()} invalid hashes")
+        invalidHashes.each { String invalidHash ->
+            logger.info("Verifying hash format: ${invalidHash}")
+            boolean validFormat = ConfigEncryptionTool.verifyHashFormat(invalidHash)
+            logger.info("Valid format: ${validFormat}")
+
+            // Assert
+            assert !validFormat
+        }
+    }
+
+    @Test
+    void testGetMigrationKeyShouldVerifySecureHashOfPassword() {
+        // Arrange
+        File bootstrapWithKeyFile = new File("src/test/resources/bootstrap_with_master_key_password.conf")
+        File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
+        bootstrapFile.delete()
+
+        Files.copy(bootstrapWithKeyFile.toPath(), bootstrapFile.toPath())
+
+        String expectedMigrationKey = bootstrapFile.readLines().find {
+            it.startsWith("nifi.bootstrap.sensitive.key=")
+        }.split("=").last()
+        logger.info("Retrieved expected migration key ${expectedMigrationKey} from bootstrap.conf")
+
+        File secureHashSourceFile = new File("src/test/resources/secure_hash.key")
+        File secureHashFile = new File("target/tmp/secure_hash.key")
+        secureHashFile.delete()
+
+        Files.copy(secureHashSourceFile.toPath(), secureHashFile.toPath())
+
+        // The second line in the file is for the password
+        String expectedHash = secureHashFile.readLines().last().split("=").last()
+        logger.info("Retrieved expected hash ${expectedHash} from secure_hash.key")
+
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+        tool.usingSecureHash = true
+        tool.secureHashPath = secureHashFile.path
+        tool.bootstrapConfPath = bootstrapFile.path
+
+        String correctHash = expectedHash
+        String incorrectHash = correctHash[0..-10] + ("x" * 9)
+
+        // Act
+        tool.secureHashPassword = correctHash
+        logger.info("Trying to retrieve migration key comparing: \n" +
+                "Command-line provided hash: ${correctHash}\n" +
+                " Hash from secure_hash.key: ${expectedHash}")
+        String correctRetrievedMigrationKey = tool.getMigrationKey()
+        logger.info("  [Correct] Retrieved migration key: ${correctRetrievedMigrationKey}")
+
+        tool.secureHashPassword = incorrectHash
+        logger.info("Trying to retrieve migration key comparing: \n" +
+                "Command-line provided hash: ${incorrectHash}\n" +
+                " Hash from secure_hash.key: ${expectedHash}")
+        def msg = shouldFail() {
+            String incorrectRetrievedMigrationKey = tool.getMigrationKey()
+            logger.info("[Incorrect] Retrieved migration key: ${incorrectRetrievedMigrationKey}")
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert correctRetrievedMigrationKey == expectedMigrationKey
+        assert msg =~ "The provided hashed key/password is not correct"
+    }
+
+    @Test
+    void testGetMigrationKeyShouldVerifySecureHashOfKey() {
+        // Arrange
+        File bootstrapWithKeyFile = new File("src/test/resources/bootstrap_with_master_key_password.conf")
+        File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
+        bootstrapFile.delete()
+
+        Files.copy(bootstrapWithKeyFile.toPath(), bootstrapFile.toPath())
+
+        String expectedMigrationKey = bootstrapFile.readLines().find {
+            it.startsWith("nifi.bootstrap.sensitive.key=")
+        }.split("=").last()
+        logger.info("Retrieved expected migration key ${expectedMigrationKey} from bootstrap.conf")
+
+        File secureHashSourceFile = new File("src/test/resources/secure_hash.key")
+        File secureHashFile = new File("target/tmp/secure_hash.key")
+        secureHashFile.delete()
+
+        Files.copy(secureHashSourceFile.toPath(), secureHashFile.toPath())
+
+        // The first line in the file is for the key
+        String expectedHash = secureHashFile.readLines().first().split("=").last()
+        logger.info("Retrieved expected hash ${expectedHash} from secure_hash.key")
+
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+        tool.usingSecureHash = true
+        tool.secureHashPath = secureHashFile.path
+        tool.bootstrapConfPath = bootstrapFile.path
+
+        String correctHash = expectedHash
+        String incorrectHash = correctHash[0..-10] + ("x" * 9)
+
+        // Act
+        tool.secureHashKey = correctHash
+        logger.info("Trying to retrieve migration key comparing: \n" +
+                "Command-line provided hash: ${correctHash}\n" +
+                " Hash from secure_hash.key: ${expectedHash}")
+        String correctRetrievedMigrationKey = tool.getMigrationKey()
+        logger.info("  [Correct] Retrieved migration key: ${correctRetrievedMigrationKey}")
+
+        tool.secureHashKey = incorrectHash
+        logger.info("Trying to retrieve migration key comparing: \n" +
+                "Command-line provided hash: ${incorrectHash}\n" +
+                " Hash from secure_hash.key: ${expectedHash}")
+        def msg = shouldFail() {
+            String incorrectRetrievedMigrationKey = tool.getMigrationKey()
+            logger.info("[Incorrect] Retrieved migration key: ${incorrectRetrievedMigrationKey}")
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert correctRetrievedMigrationKey == expectedMigrationKey
+        assert msg =~ "The provided hashed key/password is not correct"
+    }
+
     @Test
     void testShouldDecryptLoginIdentityProviders() {
         // Arrange
@@ -3463,7 +4172,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 def originalAuthorizersParsedXml = new XmlSlurper().parseText(originalAuthorizersXmlContent)
                 def updatedAuthorizersParsedXml = new XmlSlurper().parseText(updatedAuthorizersXmlContent)
                 assert originalAuthorizersParsedXml != updatedAuthorizersParsedXml
-                assert originalAuthorizersParsedXml.'**'.findAll { it.@encryption } != updatedAuthorizersParsedXml.'**'.findAll {
+                assert originalAuthorizersParsedXml.'**'.findAll {
+                    it.@encryption
+                } != updatedAuthorizersParsedXml.'**'.findAll {
                     it.@encryption
                 }
                 def authorizersEncryptedValues = updatedAuthorizersParsedXml.userGroupProvider.find {
@@ -3476,7 +4187,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 }
                 // Check that the comments are still there
                 def authorizersTrimmedLines = inputAuthorizersFile.readLines().collect { it.trim() }.findAll { it }
-                def authorizersTrimmedSerializedLines = updatedAuthorizersXmlContent.split("\n").collect { it.trim() }.findAll { it }
+                def authorizersTrimmedSerializedLines = updatedAuthorizersXmlContent.split("\n").collect {
+                    it.trim()
+                }.findAll { it }
                 assert authorizersTrimmedLines.size() == authorizersTrimmedSerializedLines.size()
 
                 /*** Bootstrap assertions ***/
@@ -3998,7 +4711,9 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
                 // Assert
                 // Get the updated nifi.properties and check the sensitive key
                 final List<String> updatedPropertiesLines = workingNiFiPropertiesFile.readLines()
-                String updatedSensitiveKeyLine = updatedPropertiesLines.find { it.startsWith(NiFiProperties.SENSITIVE_PROPS_KEY) }
+                String updatedSensitiveKeyLine = updatedPropertiesLines.find {
+                    it.startsWith(NiFiProperties.SENSITIVE_PROPS_KEY)
+                }
                 logger.info("Updated key line: ${updatedSensitiveKeyLine}")
 
                 // Check that the output values for everything are the same except the sensitive props key
@@ -4436,7 +5151,199 @@ class ConfigEncryptionToolTest extends GroovyTestCase {
         assert tool.loadFlowXml() == xmlContent
     }
 
-    // TODO: Test with 128/256-bit available
+    @Test
+    void testShouldDetectActionFlags() {
+        // Arrange
+        final def HELP_AND_VERBOSE_ARGS = [["-h", "--help"], ["-v", "--verbose"]]
+        final List<String> IGNORED_ARGS = ["currentHashParams"]
+
+        // Create a list with combinations of h[elp] and v[erbose], individual flags, and empty flag
+        def args = GroovyCollections.combinations(HELP_AND_VERBOSE_ARGS as Iterable) + HELP_AND_VERBOSE_ARGS.flatten().collect {
+            [it]
+        } + [[""]]
+        String acceptableArg = "--currentHashParams"
+        String unacceptableArg = "--migrate"
+
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+        tool.isVerbose = true
+        CommandLineParser parser = new DefaultParser()
+
+        // Act
+        args.each { List<String> invocationArgs ->
+            // Run each scenario with an allowed argument and without
+            [IGNORED_ARGS, []].each { List<String> acceptableArgs ->
+                // Check ""/-h/-v alone
+                logger.info("Checking '${invocationArgs.join(" ")}' with acceptable args: ${acceptableArgs}")
+                CommandLine commandLine = parser.parse(ConfigEncryptionTool.getCliOptions(), invocationArgs as String[])
+                boolean cleanRun = tool.commandLineHasActionFlags(commandLine, acceptableArgs)
+                logger.info("Clean run has action flags: ${cleanRun} | Expected: false")
+
+                // Check with an allowed/ignored arg
+                def allowedArgs = invocationArgs + acceptableArg
+                logger.info("Checking '${allowedArgs.join(" ")}' with acceptable args: ${acceptableArgs}")
+                commandLine = parser.parse(ConfigEncryptionTool.getCliOptions(), allowedArgs as String[])
+                boolean allowedRun = tool.commandLineHasActionFlags(commandLine, acceptableArgs)
+                logger.info("Allowed run has action flags: ${allowedRun} | Expected: ${acceptableArgs.isEmpty().toString()}")
+
+                // Check with an unallowed arg
+                def unallowedArgs = invocationArgs + unacceptableArg
+                logger.info("Checking '${unallowedArgs.join(" ")}' with acceptable args: ${acceptableArgs}")
+                commandLine = parser.parse(ConfigEncryptionTool.getCliOptions(), unallowedArgs as String[])
+                boolean unallowedRun = tool.commandLineHasActionFlags(commandLine, acceptableArgs)
+                logger.info("Unallowed run has action flags: ${unallowedRun} | Expected: true")
+
+                // Assert
+                assert !cleanRun
+                assert allowedRun == acceptableArgs.isEmpty()
+                assert unallowedRun
+            }
+        }
+    }
+
+    @Test
+    void testShouldReturnCurrentHashParams() {
+        // Arrange
+
+        // Params from secure_hash.key
+        int N = 2**4
+        int r = 8
+        int p = 1
+        String base64Salt = "A" * 22
+
+        String expectedJsonParams = new JsonBuilder([N: N, r: r, p: p, salt: base64Salt]).toString()
+        logger.info("Expected JSON params: ${expectedJsonParams}")
+
+        // Set up assertions for after System.exit()
+        exit.expectSystemExitWithStatus(0)
+
+        // Initial set up
+        File tmpDir = new File("target/tmp/")
+        tmpDir.mkdirs()
+        setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
+
+        // Copy the hashed credentials file
+        String secureHashedPasswordPath = isUnlimitedStrengthCryptoAvailable() ? "src/test/resources/secure_hash.key" :
+                "src/test/resources/secure_hash_128.key"
+        File originalSecureHashedPasswordFile = new File(secureHashedPasswordPath)
+        File secureHashedFile = new File("target/tmp/tmp_secure_hash.key")
+        secureHashedFile.delete()
+        Files.copy(originalSecureHashedPasswordFile.toPath(), secureHashedFile.toPath())
+
+        exit.checkAssertionAfterwards(new Assertion() {
+            void checkAssertion() {
+                // If JSON ordering changes, may need to capture and build JSON object from this text
+                assert systemOutRule.getLog().contains(expectedJsonParams)
+
+                // Clean up
+                tmpDir.deleteOnExit()
+                secureHashedFile.deleteOnExit()
+            }
+        })
+
+        // Override the "final" secure hash file path
+        ConfigEncryptionTool.secureHashPath = secureHashedFile.path
+
+        // Act
+        ConfigEncryptionTool.main(["--currentHashParams"] as String[])
+
+        // Assert
+
+        // Assertions defined above
+    }
+
+    @Test
+    void testShouldReturnDefaultHashParamsIfNonePresent() {
+        // Arrange
+
+        // Default params
+        int N = ConfigEncryptionTool.SCRYPT_N
+        int r = ConfigEncryptionTool.SCRYPT_R
+        int p = ConfigEncryptionTool.SCRYPT_P
+
+        String expectedJsonParams = new JsonBuilder([N: N, r: r, p: p, salt: "<some 22 char B64 str>"]).toString()
+        logger.info("Expected JSON params: ${expectedJsonParams}")
+
+        // Set up assertions for after System.exit()
+        exit.expectSystemExitWithStatus(0)
+
+        // Initial set up
+        File tmpDir = new File("target/tmp/")
+        tmpDir.mkdirs()
+        setFilePermissions(tmpDir, [PosixFilePermission.OWNER_READ, PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE, PosixFilePermission.GROUP_READ, PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE, PosixFilePermission.OTHERS_READ, PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE])
+
+        // Ensure the file is not present
+        File secureHashedFile = new File("target/tmp/tmp_secure_hash.key")
+        secureHashedFile.delete()
+
+        exit.checkAssertionAfterwards(new Assertion() {
+            void checkAssertion() {
+                // If JSON ordering changes, may need to capture and build JSON object from this text
+                List<String> returnedJSONParams = systemOutRule.getLog().readLines()
+                logger.returned("Returned JSON params: ${returnedJSONParams.join("\n")}")
+
+                JsonSlurper slurper = new JsonSlurper()
+                def expectedJson = slurper.parseText(expectedJsonParams)
+                def returnedJson = slurper.parseText(returnedJSONParams.first())
+                assert returnedJson.N == expectedJson.N
+                assert returnedJson.r == expectedJson.r
+                assert returnedJson.p == expectedJson.p
+                assert returnedJson.salt =~ /[\w\/]{22}/
+
+                // Clean up
+                tmpDir.deleteOnExit()
+            }
+        })
+
+        // Override the "final" secure hash file path
+        ConfigEncryptionTool.secureHashPath = secureHashedFile.path
+
+        // Act
+        ConfigEncryptionTool.main(["--currentHashParams"] as String[])
+
+        // Assert
+
+        // Assertions defined above
+    }
+
+    @Test
+    void testShouldFailOnCurrentHashParamsIfOtherFlagsPresent() {
+        // Arrange
+        ConfigEncryptionTool tool = new ConfigEncryptionTool()
+
+        def validOpts = [
+                "",
+                "-v",
+                "--verbose"
+        ]
+
+        def invalidOpts = [
+                "--migrate",
+                "-f flow.xml.gz",
+                "-n nifi.properties",
+                "-o output"
+        ]
+
+        // Act
+        validOpts.each { String valid ->
+            def args = (valid + " --currentHashParams").split(" ")
+            logger.info("Testing with ${args}")
+            tool.parse(args as String[])
+        }
+
+        invalidOpts.each { String invalid ->
+            def args = (invalid + " --currentHashParams").split(" ")
+            logger.info("Testing with ${args}")
+            def msg = shouldFail(CommandLineParseException) {
+                tool.parse(args as String[])
+            }
+
+            // Assert
+            assert msg == "When '--currentHashParams' is specified, only '-h'/'--help' and '-v'/'--verbose' are allowed"
+            assert systemOutRule.getLog().contains("usage: org.apache.nifi.properties.ConfigEncryptionTool [")
+        }
+    }
+
+// TODO: Test with 128/256-bit available
 }
 
 class TestAppender extends AppenderSkeleton {

http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/scrypt.py
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/scrypt.py b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/scrypt.py
new file mode 100644
index 0000000..97ba86f
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/scrypt.py
@@ -0,0 +1,18 @@
+#!/bin/env python
+
+import base64
+from passlib.hash import scrypt
+
+
+def secure_hash(password, base64_encoded_salt):
+    hash = scrypt.using(salt=base64.b64decode(base64_encoded_salt), rounds=4, block_size=8, parallelism=1).hash(password)
+    return hash
+
+
+passwords=["password", "thisIsABadPassword", "bWZerzZo6fw9ZrDz*YfM6CVj2Ktx(YJd"]
+salts=["AAAAAAAAAAAAAAAAAAAAAA==", "ABCDEFGHIJKLMNOPQRSTUV==", "eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc="]
+
+for pw in passwords:
+    for s in salts:
+        print('Hashed "{}" with salt "{}": \t{}'.format(pw, s, secure_hash(pw, s)))
+

http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash.key
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash.key b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash.key
new file mode 100644
index 0000000..ef7097e
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash.key
@@ -0,0 +1,2 @@
+secureHashKey=$s0$40801$AAAAAAAAAAAAAAAAAAAAAA$pJOGA9sPL+pRzynnwt6G2FfVTyLQdbKSbk6W8IKId8E
+secureHashPassword=$s0$40801$AAAAAAAAAAAAAAAAAAAAAA$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash_128.key
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash_128.key b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash_128.key
new file mode 100644
index 0000000..58aa040
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/secure_hash_128.key
@@ -0,0 +1,2 @@
+secureHashKey=
+secureHashPassword=
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/commandLine/ExitCode.java
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/commandLine/ExitCode.java b/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/commandLine/ExitCode.java
index 6ff733c..234faf0 100644
--- a/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/commandLine/ExitCode.java
+++ b/nifi-toolkit/nifi-toolkit-tls/src/main/java/org/apache/nifi/toolkit/tls/commandLine/ExitCode.java
@@ -69,5 +69,10 @@ public enum ExitCode {
     /**
      * Unable to read nifi.properties
      */
-    ERROR_READING_NIFI_PROPERTIES
+    ERROR_READING_NIFI_PROPERTIES,
+
+    /**
+     * Unable to read existing configuration value or file
+     */
+    ERROR_READING_CONFIG
 }


[2/2] nifi git commit: NIFI-4942 [WIP] Added skeleton for secure hash handling in encrypt-config toolkit. Added test resource for Python scrypt implementation/verifier. Added unit tests.

Posted by ym...@apache.org.
NIFI-4942 [WIP] Added skeleton for secure hash handling in encrypt-config toolkit. Added test resource for Python scrypt implementation/verifier. Added unit tests.

NIFI-4942 [WIP] More unit tests passing.

NIFI-4942 All unit tests pass and test artifacts are cleaned up.

NIFI-4942 Added RAT exclusions.

NIFI-4942 Added Scrypt hash format checker. Added unit tests.

NIFI-4942 Added NiFi hash format checker. Added unit tests.

NIFI-4942 Added check for simultaneous use of -z/-y. Added logic to check hashed password/key. Added logic to retrieve secure hash from file to compare. Added unit tests (125/125).

NIFI-4942 Added new ExitCode. Added logic to return current hash params in JSON for Ambari to consume. Fixed typos in error messages. Added unit tests (129/129).

NIFI-4942 Added Scrypt hash format verification for hash check. Added unit tests.

NIFI-4942 Fixed RAT checks.

Signed-off-by: Yolanda Davis <ym...@apache.org>

This closes #2628


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6d06defa
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6d06defa
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6d06defa

Branch: refs/heads/master
Commit: 6d06defa6336a0902180007182024219571e12e7
Parents: 82ac815
Author: Andy LoPresto <al...@apache.org>
Authored: Thu Mar 22 20:47:53 2018 -0700
Committer: Yolanda Davis <ym...@apache.org>
Committed: Fri Apr 13 18:25:09 2018 -0400

----------------------------------------------------------------------
 .../security/util/crypto/scrypt/Scrypt.java     |  43 +-
 .../util/scrypt/ScryptGroovyTest.groovy         |  56 ++
 .../nifi-toolkit-encrypt-config/pom.xml         |  15 +-
 .../nifi/properties/ConfigEncryptionTool.groovy | 373 +++++++-
 .../properties/ConfigEncryptionToolTest.groovy  | 919 ++++++++++++++++++-
 .../src/test/resources/scrypt.py                |  18 +
 .../src/test/resources/secure_hash.key          |   2 +
 .../src/test/resources/secure_hash_128.key      |   2 +
 .../nifi/toolkit/tls/commandLine/ExitCode.java  |   7 +-
 9 files changed, 1398 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/scrypt/Scrypt.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/scrypt/Scrypt.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/scrypt/Scrypt.java
index 2aeae3d..b5622f8 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/scrypt/Scrypt.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/crypto/scrypt/Scrypt.java
@@ -24,6 +24,7 @@ import java.security.GeneralSecurityException;
 import java.security.SecureRandom;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.regex.Pattern;
 import javax.crypto.Mac;
 import javax.crypto.spec.SecretKeySpec;
 import org.apache.commons.codec.binary.Base64;
@@ -51,6 +52,8 @@ public class Scrypt {
 
     private static final int DEFAULT_SALT_LENGTH = 16;
 
+    private static final Pattern SCRYPT_PATTERN = Pattern.compile("^\\$\\w{2}\\$\\w{5,}\\$[\\w\\/\\=\\+]{11,64}\\$[\\w\\/\\=\\+]{1,256}$");
+
     /**
      * Hash the supplied plaintext password and generate output in the format described
      * below:
@@ -158,12 +161,12 @@ public class Scrypt {
                 throw new IllegalArgumentException("Hash cannot be empty");
             }
 
-            String[] parts = hashed.split("\\$");
-
-            if (parts.length != 5 || !parts[1].equals("s0")) {
+            if (!verifyHashFormat(hashed)) {
                 throw new IllegalArgumentException("Hash is not properly formatted");
             }
 
+            String[] parts = hashed.split("\\$");
+
             List<Integer> splitParams = parseParameters(parts[2]);
             int n = splitParams.get(0);
             int r = splitParams.get(1);
@@ -189,6 +192,38 @@ public class Scrypt {
     }
 
     /**
+     * Returns true if the provided hash is a valid scrypt hash. Expected format:
+     * <p>
+     * {@code $s0$40801$ABCDEFGHIJKLMNOPQRSTUQ$hxU5g0eH6sRkBqcsiApI8jxvKRT+2QMCenV0GToiMQ8}
+     * <p>
+     * Components:
+     * <p>
+     * s0 -- version. Currently only "s0" is supported
+     * 40801 -- hex-encoded N, r, p parameters. {@see Scrypt#encodeParams()} for format
+     * ABCDEFGHIJKLMNOPQRSTUQ -- Base64-encoded (URL-safe, no padding) salt value.
+     * By default, 22 characters (16 bytes) but can be an arbitrary length between 11 and 64 characters (8 - 48 bytes) of random salt data
+     * hxU5g0eH6sRkBqcsiApI8jxvKRT+2QMCenV0GToiMQ8 -- the Base64-encoded (URL-safe, no padding)
+     * resulting hash component. By default, 43 characters (32 bytes) but can be an arbitrary length between 1 and MAX (depends on implementation, see RFC 7914)
+     *
+     * @param hash the hash to verify
+     * @return true if the format is acceptable
+     * @see Scrypt#formatSalt(byte[], int, int, int)
+     */
+    public static boolean verifyHashFormat(String hash) {
+        if (StringUtils.isBlank(hash)) {
+            return false;
+        }
+
+        // Currently, only version s0 is supported
+        if (!hash.startsWith("$s0$")) {
+            return false;
+        }
+
+        // Check against the pattern
+        return SCRYPT_PATTERN.matcher(hash).matches();
+    }
+
+    /**
      * Parses the individual values from the encoded params value in the modified-mcrypt format for the salt & hash.
      * <p/>
      * Example:
@@ -285,7 +320,7 @@ public class Scrypt {
             logger.warn("An empty salt was used for scrypt key derivation");
 //            throw new IllegalArgumentException("Salt cannot be empty");
             // as the Exception is not being thrown, prevent NPE if salt is null by setting it to empty array
-            if( salt == null ) salt = new byte[]{};
+            if (salt == null) salt = new byte[]{};
         }
 
         if (saltLength < 8 || saltLength > 32) {

http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/scrypt/ScryptGroovyTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/scrypt/ScryptGroovyTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/scrypt/ScryptGroovyTest.groovy
index 416d670..ad222a0 100644
--- a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/scrypt/ScryptGroovyTest.groovy
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/scrypt/ScryptGroovyTest.groovy
@@ -397,4 +397,60 @@ class ScryptGroovyTest {
             assert msg =~ "Hash cannot be empty|Hash is not properly formatted"
         }
     }
+
+    @Test
+    void testVerifyHashFormatShouldDetectValidHash() throws Exception {
+        // Arrange
+        final def VALID_HASHES = [
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$hxU5g0eH6sRkBqcsiApI8jxvKRT+2QMCenV0GToiMQ8",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$99aTTB39TJo69aZCONQmRdyWOgYsDi+1MI+8D0EgMNM",
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$Gk7K9YmlsWbd8FS7e4RKVWnkg9vlsqYnlD593pJ71gg",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$Ri78VZbrp2cCVmGh2a9Nbfdov8LPnFb49MYyzPCaXmE",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$rZIrP2qdIY7LN4CZAMgbCzl3YhXz6WhaNyXJXqFIjaI",
+                "\$s0\$40801\$AAAAAAAAAAAAAAAAAAAAAA\$GxH68bGykmPDZ6gaPIGOONOT2omlZ7cd0xlcZ9UsY/0",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPQRSTUQ\$KLGZjWlo59sbCbtmTg5b4k0Nu+biWZRRzhPhN7K5kkI",
+                "\$s0\$40801\$eO+UUcKYL2gnpD51QCc+gnywQ7Eg9tZeLMlf0XXr2zc\$6Ql6Efd2ac44ERoV31CL3Q0J3LffNZKN4elyMHux99Y",
+                // Uncommon but technically valid
+                "\$s0\$F0801\$AAAAAAAAAAA\$A",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOP\$A",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOP\$ABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOP",
+                "\$s0\$40801\$ABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOP\$ABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOPABCDEFGHIJKLMNOP",
+                "\$s0\$F0801\$AAAAAAAAAAA\$A",
+                "\$s0\$F0801\$AAAAAAAAAAA\$A",
+                "\$s0\$F0801\$AAAAAAAAAAA\$A",
+                "\$s0\$F0801\$AAAAAAAAAAA\$A",
+                "\$s0\$F0801\$AAAAAAAAAAA\$A",
+        ]
+
+        // Act
+        VALID_HASHES.each { String validHash ->
+            logger.info("Using hash: ${validHash}")
+
+            boolean isValidHash = Scrypt.verifyHashFormat(validHash)
+            logger.info("Hash is valid: ${isValidHash}")
+
+            // Assert
+            assert isValidHash
+        }
+    }
+
+    @Test
+    void testVerifyHashFormatShouldDetectInvalidHash() throws Exception {
+        // Arrange
+
+        // Even though the spec allows for empty salts, the JCE does not, so extend enforcement of that to the user boundary
+        final def INVALID_HASHES = ['', null, '$s0$a0801$', '$s0$a0801$abcdefghijklmnopqrstuv$']
+
+        // Act
+        INVALID_HASHES.each { String invalidHash ->
+            logger.info("Using hash: ${invalidHash}")
+
+            boolean isValidHash = Scrypt.verifyHashFormat(invalidHash)
+            logger.info("Hash is valid: ${isValidHash}")
+
+            // Assert
+            assert !isValidHash
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml b/nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml
index 0bf1f74..2163de6 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/pom.xml
@@ -13,7 +13,8 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <parent>
         <groupId>org.apache.nifi</groupId>
         <artifactId>nifi-toolkit</artifactId>
@@ -162,7 +163,19 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludes combine.children="append">
+                        <exclude>src/test/resources/scrypt.py</exclude>
+                        <exclude>src/test/resources/secure_hash.key</exclude>
+                        <exclude>src/test/resources/secure_hash_128.key</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
         </plugins>
+
     </build>
 
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/6d06defa/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy
index 0e507c8..e8ac642 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/properties/ConfigEncryptionTool.groovy
@@ -17,6 +17,7 @@
 package org.apache.nifi.properties
 
 import groovy.io.GroovyPrintWriter
+import groovy.json.JsonBuilder
 import groovy.xml.XmlUtil
 import org.apache.commons.cli.CommandLine
 import org.apache.commons.cli.CommandLineParser
@@ -27,6 +28,8 @@ import org.apache.commons.cli.Options
 import org.apache.commons.cli.ParseException
 import org.apache.commons.codec.binary.Hex
 import org.apache.commons.io.IOUtils
+import org.apache.nifi.security.util.crypto.CipherUtility
+import org.apache.nifi.security.util.crypto.scrypt.Scrypt
 import org.apache.nifi.toolkit.tls.commandLine.CommandLineParseException
 import org.apache.nifi.toolkit.tls.commandLine.ExitCode
 import org.apache.nifi.util.NiFiProperties
@@ -44,7 +47,9 @@ import javax.crypto.SecretKeyFactory
 import javax.crypto.spec.PBEKeySpec
 import javax.crypto.spec.PBEParameterSpec
 import java.nio.charset.StandardCharsets
+import java.security.InvalidKeyException
 import java.security.KeyException
+import java.security.MessageDigest
 import java.security.SecureRandom
 import java.security.Security
 import java.util.zip.GZIPInputStream
@@ -62,11 +67,15 @@ class ConfigEncryptionTool {
     public String outputAuthorizersPath
     public String flowXmlPath
     public String outputFlowXmlPath
+    // If this value can be set by the running user, it can point to a manipulated file anywhere
+    private static String secureHashPath = "./secure_hash.key"
 
     private String keyHex
     private String migrationKeyHex
     private String password
     private String migrationPassword
+    private String secureHashKey
+    private String secureHashPassword
 
     // This is the raw value used in nifi.sensitive.props.key
     private String flowPropertiesPassword
@@ -81,6 +90,7 @@ class ConfigEncryptionTool {
 
     private boolean usingPassword = true
     private boolean usingPasswordMigration = true
+    private boolean usingSecureHash = false
     private boolean migration = false
     private boolean isVerbose = false
     private boolean handlingNiFiProperties = false
@@ -88,6 +98,7 @@ class ConfigEncryptionTool {
     private boolean handlingAuthorizers = false
     private boolean handlingFlowXml = false
     private boolean ignorePropertiesFiles = false
+    private boolean queryingCurrentHashParams = false
 
     private static final String HELP_ARG = "help"
     private static final String VERBOSE_ARG = "verbose"
@@ -104,21 +115,30 @@ class ConfigEncryptionTool {
     private static final String PASSWORD_ARG = "password"
     private static final String KEY_MIGRATION_ARG = "oldKey"
     private static final String PASSWORD_MIGRATION_ARG = "oldPassword"
+    private static final String HASHED_KEY_MIGRATION_ARG = "secureHashKey"
+    private static final String HASHED_PASSWORD_MIGRATION_ARG = "secureHashPassword"
     private static final String USE_KEY_ARG = "useRawKey"
     private static final String MIGRATION_ARG = "migrate"
     private static final String PROPS_KEY_ARG = "propsKey"
     private static final String DO_NOT_ENCRYPT_NIFI_PROPERTIES_ARG = "encryptFlowXmlOnly"
     private static final String NEW_FLOW_ALGORITHM_ARG = "newFlowAlgorithm"
     private static final String NEW_FLOW_PROVIDER_ARG = "newFlowProvider"
+    private static final String CURRENT_HASH_PARAMS_ARG = "currentHashParams"
+
+    // Static holder to avoid re-generating the options object multiple times in an invocation
+    private static Options staticOptions
 
     // Hard-coded fallback value from {@link org.apache.nifi.encrypt.StringEncryptor}
     private static final String DEFAULT_NIFI_SENSITIVE_PROPS_KEY = "nififtw!"
     private static final int MIN_PASSWORD_LENGTH = 12
 
-    // Strong parameters as of 12 Aug 2016
+    // Strong parameters as of 12 Aug 2016 (for key derivation)
+    // This value can remain an int until best practice specifies a value above 2**32
     private static final int SCRYPT_N = 2**16
     private static final int SCRYPT_R = 8
     private static final int SCRYPT_P = 1
+    static final String CURRENT_SCRYPT_VERSION = "s0"
+    private static final String NIFI_SCRYPT_PATTERN = /^\$\w{2}\$\w{5,}\$[\w\/\=\+]{22,}\$[\w\/\=\+]{43}$/
 
     // Hard-coded values from StandardPBEByteEncryptor which will be removed during refactor of all flow encryption code in NIFI-1465
     private static final int DEFAULT_KDF_ITERATIONS = 1000
@@ -142,7 +162,8 @@ class ConfigEncryptionTool {
             "files or in flow.xml.gz to be encrypted with a new key."
 
     private static final String LDAP_PROVIDER_CLASS = "org.apache.nifi.ldap.LdapProvider"
-    private static final String LDAP_PROVIDER_REGEX = /(?s)<provider>(?:(?!<provider>).)*?<class>\s*org\.apache\.nifi\.ldap\.LdapProvider.*?<\/provider>/
+    private static
+    final String LDAP_PROVIDER_REGEX = /(?s)<provider>(?:(?!<provider>).)*?<class>\s*org\.apache\.nifi\.ldap\.LdapProvider.*?<\/provider>/
     /* Explanation of LDAP_PROVIDER_REGEX:
      *   (?s)                             -> single-line mode (i.e., `.` in regex matches newlines)
      *   <provider>                       -> find occurrence of `<provider>` literally (case-sensitive)
@@ -177,6 +198,7 @@ class ConfigEncryptionTool {
 
     private static final String DEFAULT_PROVIDER = BouncyCastleProvider.PROVIDER_NAME
     private static final String DEFAULT_FLOW_ALGORITHM = "PBEWITHMD5AND256BITAES-CBC-OPENSSL"
+    static private final int AMBARI_COMPATIBLE_SCRYPT_HASH_LENGTH = 256
 
     private static String buildHeader(String description = DEFAULT_DESCRIPTION) {
         "${SEP}${description}${SEP * 2}"
@@ -196,7 +218,11 @@ class ConfigEncryptionTool {
 
     ConfigEncryptionTool(String description) {
         this.header = buildHeader(description)
-        this.options = new Options()
+        this.options = getCliOptions()
+    }
+
+    static Options buildOptions() {
+        Options options = new Options()
         options.addOption(Option.builder("h").longOpt(HELP_ARG).hasArg(false).desc("Show usage information (this message)").build())
         options.addOption(Option.builder("v").longOpt(VERBOSE_ARG).hasArg(false).desc("Sets verbose mode (default false)").build())
         options.addOption(Option.builder("n").longOpt(NIFI_PROPERTIES_ARG).hasArg(true).argName("file").desc("The nifi.properties file containing unprotected config values (will be overwritten unless -o is specified)").build())
@@ -212,23 +238,30 @@ class ConfigEncryptionTool {
         options.addOption(Option.builder("e").longOpt(KEY_MIGRATION_ARG).hasArg(true).argName("keyhex").desc("The old raw hexadecimal key to use during key migration").build())
         options.addOption(Option.builder("p").longOpt(PASSWORD_ARG).hasArg(true).argName("password").desc("The password from which to derive the key to use to encrypt the sensitive properties").build())
         options.addOption(Option.builder("w").longOpt(PASSWORD_MIGRATION_ARG).hasArg(true).argName("password").desc("The old password from which to derive the key during migration").build())
+        options.addOption(Option.builder("y").longOpt(HASHED_KEY_MIGRATION_ARG).hasArg(true).argName("hashed_keyhex").desc("The old securely-hashed hexadecimal key to authenticate during key migration (see NiFi Admin Guide)").build())
+        options.addOption(Option.builder("z").longOpt(HASHED_PASSWORD_MIGRATION_ARG).hasArg(true).argName("hashed_password").desc("The old securely-hashed password to authenticate during key migration (see NiFi Admin Guide)").build())
         options.addOption(Option.builder("r").longOpt(USE_KEY_ARG).hasArg(false).desc("If provided, the secure console will prompt for the raw key value in hexadecimal form").build())
         options.addOption(Option.builder("m").longOpt(MIGRATION_ARG).hasArg(false).desc("If provided, the nifi.properties and/or login-identity-providers.xml sensitive properties will be re-encrypted with a new key").build())
         options.addOption(Option.builder("x").longOpt(DO_NOT_ENCRYPT_NIFI_PROPERTIES_ARG).hasArg(false).desc("If provided, the properties in flow.xml.gz will be re-encrypted with a new key but the nifi.properties and/or login-identity-providers.xml files will not be modified").build())
         options.addOption(Option.builder("s").longOpt(PROPS_KEY_ARG).hasArg(true).argName("password|keyhex").desc("The password or key to use to encrypt the sensitive processor properties in flow.xml.gz").build())
         options.addOption(Option.builder("A").longOpt(NEW_FLOW_ALGORITHM_ARG).hasArg(true).argName("algorithm").desc("The algorithm to use to encrypt the sensitive processor properties in flow.xml.gz").build())
         options.addOption(Option.builder("P").longOpt(NEW_FLOW_PROVIDER_ARG).hasArg(true).argName("algorithm").desc("The security provider to use to encrypt the sensitive processor properties in flow.xml.gz").build())
+        options.addOption(Option.builder().longOpt(CURRENT_HASH_PARAMS_ARG).hasArg(false).desc("Returns the current salt and cost params used to store the hashed key/password").build())
+        options
     }
 
     static Options getCliOptions() {
-        return new ConfigEncryptionTool().options
+        if (!staticOptions) {
+            staticOptions = buildOptions()
+        }
+        return staticOptions
     }
 
-    /**
-     * Prints the usage message and available arguments for this tool (along with a specific error message if provided).
-     *
-     * @param errorMessage the optional error message
-     */
+/**
+ * Prints the usage message and available arguments for this tool (along with a specific error message if provided).
+ *
+ * @param errorMessage the optional error message
+ */
     void printUsage(String errorMessage) {
         if (errorMessage) {
             System.out.println(errorMessage)
@@ -236,7 +269,8 @@ class ConfigEncryptionTool {
         }
         HelpFormatter helpFormatter = new HelpFormatter()
         helpFormatter.setWidth(160)
-        helpFormatter.setOptionComparator(null) // preserve manual ordering of options when printing instead of alphabetical
+        helpFormatter.setOptionComparator(null)
+        // preserve manual ordering of options when printing instead of alphabetical
         helpFormatter.printHelp(ConfigEncryptionTool.class.getCanonicalName(), header, options, FOOTER, true)
     }
 
@@ -257,6 +291,17 @@ class ConfigEncryptionTool {
 
             isVerbose = commandLine.hasOption(VERBOSE_ARG)
 
+            // If this flag is present, ensure no other options are present and then fail/return
+            if (commandLine.hasOption(CURRENT_HASH_PARAMS_ARG)) {
+                queryingCurrentHashParams = true
+                if (commandLineHasActionFlags(commandLine, [CURRENT_HASH_PARAMS_ARG])) {
+                    printUsageAndThrow("When '--${CURRENT_HASH_PARAMS_ARG}' is specified, only '-h'/'--${HELP_ARG}' and '-v'/'--${VERBOSE_ARG}' are allowed", ExitCode.INVALID_ARGS)
+                } else {
+                    // Otherwise return (avoid unnecessary parsing)
+                    return commandLine
+                }
+            }
+
             bootstrapConfPath = commandLine.getOptionValue(BOOTSTRAP_CONF_ARG)
 
             // If this flag is provided, the nifi.properties is necessary to read/write the flow encryption key, but the encryption process will not actually be applied to nifi.properties / login-identity-providers.xml
@@ -361,6 +406,29 @@ class ConfigEncryptionTool {
                 if (isVerbose) {
                     logger.info("Key migration mode activated")
                 }
+                if (isSecureHashArgumentPresent(commandLine)) {
+                    logger.info("Secure hash argument present")
+
+                    // Check for old key/password and throw error
+                    if (commandLine.hasOption(KEY_MIGRATION_ARG) || commandLine.hasOption(PASSWORD_MIGRATION_ARG)) {
+                        printUsageAndThrow("If the '-w'/'--${PASSWORD_MIGRATION_ARG}' or '-e'/'--${KEY_MIGRATION_ARG}' arguments are present, '-z'/'--${HASHED_PASSWORD_MIGRATION_ARG}' and '-y'/'--${HASHED_KEY_MIGRATION_ARG}' cannot be used", ExitCode.INVALID_ARGS)
+                    }
+
+                    // Check for both key and password and throw error
+                    if (commandLine.hasOption(HASHED_KEY_MIGRATION_ARG) && commandLine.hasOption(HASHED_PASSWORD_MIGRATION_ARG)) {
+                        printUsageAndThrow("Only one of '-z'/'--${HASHED_PASSWORD_MIGRATION_ARG}' and '-y'/'--${HASHED_KEY_MIGRATION_ARG}' can be used together", ExitCode.INVALID_ARGS)
+                    }
+
+                    // Extract flags to field
+                    if (commandLine.hasOption(HASHED_KEY_MIGRATION_ARG)) {
+                        secureHashKey = commandLine.getOptionValue(HASHED_KEY_MIGRATION_ARG)
+                    } else {
+                        secureHashPassword = commandLine.getOptionValue(HASHED_PASSWORD_MIGRATION_ARG)
+                    }
+
+                    // Set boolean flag to true
+                    usingSecureHash = true
+                }
                 if (commandLine.hasOption(PASSWORD_MIGRATION_ARG)) {
                     usingPasswordMigration = true
                     if (commandLine.hasOption(KEY_MIGRATION_ARG)) {
@@ -370,7 +438,8 @@ class ConfigEncryptionTool {
                     }
                 } else {
                     migrationKeyHex = commandLine.getOptionValue(KEY_MIGRATION_ARG)
-                    usingPasswordMigration = !migrationKeyHex
+                    // Use the "migration password" value if the migration key hex is absent and the secure hash password/key hex is absent (if either are present, the migration password is not)
+                    usingPasswordMigration = !migrationKeyHex && !usingSecureHash
                 }
             } else {
                 if (commandLine.hasOption(PASSWORD_MIGRATION_ARG) || commandLine.hasOption(KEY_MIGRATION_ARG)) {
@@ -410,15 +479,43 @@ class ConfigEncryptionTool {
         return commandLine
     }
 
-    /**
-     * The method returns the provided, derived, or securely-entered key in hex format. The reason the parameters must be provided instead of read from the fields is because this is used for the regular key/password and the migration key/password.
-     *
-     * @param device
-     * @param keyHex
-     * @param password
-     * @param usingPassword
-     * @return
-     */
+    boolean commandLineHasActionFlags(CommandLine commandLine, List<String> acceptableOptionStrings = []) {
+        // Resolve the list of Option objects corresponding to "help" and "verbose"
+        final List<Option> ALWAYS_ACCEPTABLE_OPTIONS = resolveOptions([HELP_ARG, VERBOSE_ARG])
+
+        // Resolve the list of Option objects corresponding to the provided "additional acceptable options"
+        List<Option> acceptableOptions = resolveOptions(acceptableOptionStrings)
+
+        // Determine the options submitted to the command line that are not acceptable
+        List<Option> invalidOptions = commandLine.options - (acceptableOptions + ALWAYS_ACCEPTABLE_OPTIONS)
+        if (invalidOptions) {
+            if (isVerbose) {
+                logger.error("In this mode, the following options are invalid: ${invalidOptions}")
+            }
+            return true
+        } else {
+            return false
+        }
+    }
+
+    static List<Option> resolveOptions(List<String> strings) {
+        strings?.collect { String opt ->
+            getCliOptions().options.find { it.opt == opt || it.longOpt == opt }
+        }
+    }
+
+    static boolean isSecureHashArgumentPresent(CommandLine commandLine) {
+        commandLine.hasOption(HASHED_PASSWORD_MIGRATION_ARG) || commandLine.hasOption(HASHED_KEY_MIGRATION_ARG)
+    }
+/**
+ * The method returns the provided, derived, or securely-entered key in hex format. The reason the parameters must be provided instead of read from the fields is because this is used for the regular key/password and the migration key/password.
+ *
+ * @param device
+ * @param keyHex
+ * @param password
+ * @param usingPassword
+ * @return
+ */
     private String getKeyInternal(TextDevice device = TextDevices.defaultTextDevice(), String keyHex, String password, boolean usingPassword) {
         if (usingPassword) {
             if (!password) {
@@ -446,10 +543,89 @@ class ConfigEncryptionTool {
     }
 
     private String getMigrationKey() {
-        getKeyInternal(TextDevices.defaultTextDevice(), migrationKeyHex, migrationPassword, usingPasswordMigration)
+        if (usingSecureHash) {
+            // The boolean flag for "key" means the expression should evaluate to true when key is present and password is not
+            String knownHashValue = readSecureHashValueFromFile(secureHashKey && !secureHashPassword)
+            if (checkHashedValue(knownHashValue, getProvidedSecureHashValue())) {
+                // Retrieve the key from bootstrap.conf because the caller only has the hashed version available
+                return readMasterKeyFromBootstrap()
+            } else {
+                throw new InvalidKeyException("The provided hashed key/password is not correct")
+            }
+        } else {
+            return getKeyInternal(TextDevices.defaultTextDevice(), migrationKeyHex, migrationPassword, usingPasswordMigration)
+        }
+    }
+
+    private String getProvidedSecureHashValue() {
+        if (usingSecureHash) {
+            return secureHashPassword ?: secureHashKey
+        } else {
+            return ""
+        }
+    }
+
+    private static String readSecureHashValueFromFile(boolean readKey = true) {
+        File secureHashFile = new File(secureHashPath)
+        if (!secureHashFile.canRead()) {
+            throw new IOException("Cannot read from secure hash file")
+        }
+        List<String> lines = secureHashFile.readLines()
+        String linePrefix = readKey ? "secureHashKey" : "secureHashPassword"
+        String relevantLine = lines.find { it.startsWith(linePrefix) }
+        String hashValue = relevantLine?.split("=")?.last()
+        if (!hashValue) {
+            throw new InvalidKeyException("The secure hash of the ${readKey ? "key" : "password"} could not be read from the stored file")
+        } else {
+            return hashValue
+        }
+    }
+
+    /**
+     * Returns true if the hash values are equivalent. Currently performs a *constant-time equality* check on the two values. As the scrypt format does not allow for reversing, two "equivalent" but non-identical hash values cannot be compared for equality.
+     *
+     * Example (byte equivalent):
+     *
+     * KHV: {@code $s0$40801$AAAAAAAAAAAAAAAAAAAAAA$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM}
+     * UHV: {@code $s0$40801$AAAAAAAAAAAAAAAAAAAAAA$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM}
+     * Result: EQUAL
+     *
+     * Example (semantically equivalent):
+     *
+     * KHV: {@code $s0$40801$AAAAAAAAAAAAAAAAAAAAAA$gLSh7ChbHdOIMvZ74XGjV6qF65d9qvQ8n75FeGnM8YM}
+     * UHV: {@code $s0$40801$ABCDEFGHIJKLMNOPQRSTUQ$hxU5g0eH6sRkBqcsiApI8jxvKRT+2QMCenV0GToiMQ8}
+     * Result: NOT EQUAL
+     *
+     * Even though both hash values are the result of the input "password", by design the hash values cannot be reversed to a common origin to determine their equality. If the raw input was known, both hashes could be determined to be valid, thus asserting the correctness of the raw input and the functional equivalence of the hash values.
+     *
+     * @param knownHashValue
+     * @param unknownHashValue
+     * @return true if the hash values are present and equal
+     */
+    static boolean checkHashedValue(String knownHashValue, String unknownHashValue) {
+        if (!knownHashValue || !unknownHashValue) {
+            return false
+        }
+
+        // The values should be in scrypt format
+        if (!verifyHashFormat(knownHashValue) || !verifyHashFormat(unknownHashValue)) {
+            return false
+        }
+
+        return MessageDigest.isEqual(knownHashValue.getBytes(StandardCharsets.UTF_8), unknownHashValue.getBytes(StandardCharsets.UTF_8))
+    }
+
+    /**
+     * Returns true if the provided hash is in the valid format for NiFi secured hash storage. NiFi enforces additional constraints over the minimum Scrypt requirements (16+ byte [22+ B64] salt, 32 byte [43 B64] hash).
+     *
+     * @param hash the hash to verify
+     * @return true if the format is acceptable
+     */
+    static boolean verifyHashFormat(String hash) {
+        hash =~ NIFI_SCRYPT_PATTERN
     }
 
-    private String getFlowPassword(TextDevice textDevice = TextDevices.defaultTextDevice()) {
+    private static String getFlowPassword(TextDevice textDevice = TextDevices.defaultTextDevice()) {
         readPasswordFromConsole(textDevice)
     }
 
@@ -463,6 +639,10 @@ class ConfigEncryptionTool {
         new String(textDevice.readPassword())
     }
 
+    private String readMasterKeyFromBootstrap() {
+        NiFiPropertiesLoader.extractKeyFromBootstrapFile(bootstrapConfPath)
+    }
+
     /**
      * Returns the key in uppercase hexadecimal format with delimiters (spaces, '-', etc.) removed. All non-hex chars are removed. If the result is not a valid length (32, 48, 64 chars depending on the JCE), an exception is thrown.
      *
@@ -831,7 +1011,9 @@ class ConfigEncryptionTool {
         try {
             def doc = new XmlSlurper().parseText(encryptedXml)
             // Find the provider element by class even if it has been renamed
-            def passwords = doc.userGroupProvider.find { it.'class' as String == LDAP_USER_GROUP_PROVIDER_CLASS }.property.findAll {
+            def passwords = doc.userGroupProvider.find {
+                it.'class' as String == LDAP_USER_GROUP_PROVIDER_CLASS
+            }.property.findAll {
                 it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
             }
 
@@ -1167,6 +1349,41 @@ class ConfigEncryptionTool {
         }
     }
 
+    /**
+     * Writes the contents of the secure hash configuration file (hashed value of current migration password/key hex) to the output {@code secure_hash.key} file.
+     *
+     * @throw IOException if there is a problem reading or writing the secure_hash.key file
+     */
+    private void writeSecureHash() throws IOException {
+        if (!secureHashPath) {
+            throw new IllegalArgumentException("Cannot write hashed password/key to empty secure_hash.key path")
+        }
+
+        File secureHashFile = new File(secureHashPath)
+
+        if (isSafeToWrite(secureHashFile)) {
+            try {
+                List<String> secureHashFileLines = []
+                // Calculate the secure hash of the current key (and password if provided) using current default values for cost params and random salt
+                String secureHashKey = secureHashKey(keyHex)
+                secureHashFileLines << "secureHashKey=${secureHashKey}"
+                if (password) {
+                    String secureHashPassword = secureHashPassword(password)
+                    secureHashFileLines << "secureHashPassword=${secureHashPassword}"
+                }
+
+                // Write the updated values back to the file
+                secureHashFile.text = secureHashFileLines.join("\n")
+            } catch (IOException e) {
+                def msg = "Encountered an exception updating the secure_hash.key file with the hashed value(s)"
+                logger.error(msg, e)
+                throw e
+            }
+        } else {
+            throw new IOException("The secure_hash.key file at ${secureHashPath} must be writable by the user running this tool")
+        }
+    }
+
     private
     static List<String> serializeNiFiPropertiesAndPreserveFormat(NiFiProperties niFiProperties, File originalPropertiesFile) {
         List<String> lines = originalPropertiesFile.readLines()
@@ -1277,13 +1494,17 @@ class ConfigEncryptionTool {
         }
 
         // Generate a 128 bit salt
-        byte[] salt = generateScryptSalt()
+        byte[] salt = generateScryptSaltForKeyDerivation()
         int keyLengthInBytes = getValidKeyLengths().max() / 8
         byte[] derivedKeyBytes = SCrypt.generate(password.getBytes(StandardCharsets.UTF_8), salt, SCRYPT_N, SCRYPT_R, SCRYPT_P, keyLengthInBytes)
         Hex.encodeHexString(derivedKeyBytes).toUpperCase()
     }
 
-    private static byte[] generateScryptSalt() {
+    /**
+     * Returns a static "raw" salt (the 128 bits of random data used when generating the hash, not the "complete" {@code $s0$e0101$ABCDEFGHIJKLMNOPQRSTUV} salt format).
+     * @return the raw salt in byte[] form
+     */
+    private static byte[] generateScryptSaltForKeyDerivation() {
 //        byte[] salt = new byte[16]
 //        new SecureRandom().nextBytes(salt)
 //        salt
@@ -1294,6 +1515,32 @@ class ConfigEncryptionTool {
         "NIFI_SCRYPT_SALT".getBytes(StandardCharsets.UTF_8)
     }
 
+    private static byte[] generateRandomSalt(int bits = 128) {
+        byte[] salt = new byte[bits / 8]
+        new SecureRandom().nextBytes(salt)
+        salt
+    }
+
+    /**
+     * Generates an scrypt salt using the provided parameters and encodes it in the proper format. If a value is not provided, the listed default will be used.
+     *
+     * @param rawSalt the raw 128 bit salt to use (default is to generate a random value)
+     * @param n the CPU/memory cost factor (default is {@value ConfigEncryptionTool#SCRYPT_N})
+     * @param r the blocksize factor (default is {@value ConfigEncryptionTool#SCRYPT_R})
+     * @param p the parallelization factor (default is {@value ConfigEncryptionTool#SCRYPT_P})
+     * @param version the salt version identifier (default is {@value ConfigEncryptionTool#CURRENT_SCRYPT_VERSION})
+     * @return the scrypt-formatted salt (e.g. {@code $s0$e0101$ABCDEFGHIJKLMNOPQRSTUV})
+     */
+    private
+    static String generateScryptSalt(byte[] rawSalt = generateRandomSalt(128), int n = SCRYPT_N, int r = SCRYPT_R, int p = SCRYPT_P, String version = CURRENT_SCRYPT_VERSION) {
+        String formattedSalt = Scrypt.formatSalt(rawSalt, n, r, p)
+        def versionString = "\$${version}\$"
+        if (!formattedSalt.startsWith(versionString)) {
+            formattedSalt = formattedSalt.replaceFirst(/$\w{2}$/, versionString)
+        }
+        return formattedSalt
+    }
+
     private String getExistingFlowPassword() {
         return niFiProperties.getProperty(NiFiProperties.SENSITIVE_PROPS_KEY) as String ?: DEFAULT_NIFI_SENSITIVE_PROPS_KEY
     }
@@ -1316,6 +1563,73 @@ class ConfigEncryptionTool {
         }
     }
 
+    static String secureHashPassword(String password, String salt = generateScryptSalt()) {
+        // If empty, generate complete salt
+        if (!salt) {
+            salt = generateScryptSalt()
+        }
+
+        // The findAll() drops any empty elements
+        def saltComponents = salt.split('\\$').findAll()
+        if (saltComponents.size() < 3) {
+            throw new IllegalArgumentException("The provided salt was not in valid scrypt format")
+        }
+        if (saltComponents.first() != CURRENT_SCRYPT_VERSION) {
+            throw new IllegalArgumentException("Currently, only scrypt hashes with version ${CURRENT_SCRYPT_VERSION} are supported")
+        }
+
+        // Split the encoded params into N, R, P
+        List<Integer> params = Scrypt.parseParameters(saltComponents[1])
+
+        // Generate the hashed format
+        Scrypt.scrypt(password, Base64.decoder.decode(saltComponents[2]), params[0], params[1], params[2], AMBARI_COMPATIBLE_SCRYPT_HASH_LENGTH)
+    }
+
+    static String secureHashKey(String keyHex, String salt = generateScryptSalt()) {
+        // Lowercase the key hex, then call secureHashPassword() as the algorithm is the same
+        secureHashPassword(keyHex?.toLowerCase(), salt)
+    }
+
+
+    private String getCurrentHashParams() {
+        try {
+            int N
+            int r
+            int p
+            String base64Salt
+
+            try {
+                String secureHash = readSecureHashValueFromFile()
+
+                // The findAll() drops any empty elements
+                def hashComponents = secureHash.split('\\$').findAll()
+                if (hashComponents.size() < 3) {
+                    throw new IllegalArgumentException("The provided secure hash was not in valid scrypt format")
+                }
+                if (hashComponents.first() != CURRENT_SCRYPT_VERSION) {
+                    throw new IllegalArgumentException("Currently, only scrypt hashes with version ${CURRENT_SCRYPT_VERSION} are supported")
+                }
+
+                // Split the encoded params into N, R, P
+                List<Integer> params = Scrypt.parseParameters(hashComponents[1])
+                N = params[0]
+                r = params[1]
+                p = params[2]
+                base64Salt = hashComponents[2]
+            } catch (IOException | InvalidKeyException e) {
+                // These exceptions occur if the file doesn't exist, can't be read, or doesn't have secure hashes populated
+                N = SCRYPT_N
+                r = SCRYPT_R
+                p = SCRYPT_P
+                base64Salt = CipherUtility.encodeBase64NoPadding(generateRandomSalt())
+            }
+            return new JsonBuilder([N: N, r: r, p: p, salt: base64Salt]).toString()
+        } catch (Exception e) {
+            logger.error("Encountered an exception getting current hash parameters: ${e.getLocalizedMessage()}")
+            printUsageAndThrow(e.getLocalizedMessage(), ExitCode.ERROR_READING_CONFIG)
+        }
+    }
+
     /**
      * Runs main tool logic (parsing arguments, reading files, protecting properties, and writing key and properties out to destination files).
      *
@@ -1330,6 +1644,12 @@ class ConfigEncryptionTool {
             try {
                 tool.parse(args)
 
+                // Ensure the only content written to STDOUT is the JSON (consumable by other processes)
+                if (tool.queryingCurrentHashParams) {
+                    System.out.println(tool.getCurrentHashParams())
+                    System.exit(ExitCode.SUCCESS.ordinal())
+                }
+
                 boolean existingNiFiPropertiesAreEncrypted = tool.niFiPropertiesAreEncrypted()
                 if (!tool.ignorePropertiesFiles || (tool.handlingFlowXml && existingNiFiPropertiesAreEncrypted)) {
                     // If we are handling the flow.xml.gz and nifi.properties is already encrypted, try getting the key from bootstrap.conf rather than the console
@@ -1471,6 +1791,9 @@ class ConfigEncryptionTool {
                 synchronized (this) {
                     if (!tool.ignorePropertiesFiles) {
                         tool.writeKeyToBootstrapConf()
+
+                        // Always write the secure hash in case the next invocation needs it
+                        tool.writeSecureHash()
                     }
                     if (tool.handlingFlowXml) {
                         tool.writeFlowXmlToFile(tool.flowXml)