You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2018/01/08 19:20:05 UTC

[2/4] nifi git commit: NIFI-4708 Add Registry support to encrypt-config. Adds support for NiFI Registry config files to the encrypt-config tool in NiFi Toolkit. Also adds decryption capability to encrypt-config tool.

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy
new file mode 100644
index 0000000..0616a66
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/TestUtil.groovy
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.toolkit.encryptconfig
+
+import groovy.util.slurpersupport.GPathResult
+import org.apache.commons.lang3.SystemUtils
+import org.apache.nifi.properties.AESSensitivePropertyProvider
+import org.apache.nifi.toolkit.encryptconfig.util.NiFiRegistryAuthorizersXmlEncryptor
+import org.apache.nifi.toolkit.encryptconfig.util.NiFiRegistryIdentityProvidersXmlEncryptor
+
+import javax.crypto.Cipher
+import java.nio.file.Files
+import java.nio.file.attribute.PosixFilePermission
+
+class TestUtil {
+
+    static final String RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT = absolutePathForResource('/nifi-registry/bootstrap_default.conf')
+    static final String RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY = absolutePathForResource('/nifi-registry/bootstrap_without_master_key.conf')
+    static final String RESOURCE_REGISTRY_BOOTSTRAP_EMPTY_KEY = absolutePathForResource('/nifi-registry/bootstrap_with_empty_master_key.conf')
+    static final String RESOURCE_REGISTRY_BOOTSTRAP_KEY_128 = absolutePathForResource('/nifi-registry/bootstrap_with_master_key_128.conf')
+    static final String RESOURCE_REGISTRY_BOOTSTRAP_KEY_FROM_PASSWORD_128 = absolutePathForResource('/nifi-registry/bootstrap_with_master_key_from_password_128.conf')
+
+    static final String RESOURCE_REGISTRY_PROPERTIES_COMMENTED = absolutePathForResource('/nifi-registry/nifi-registry-commented.properties')
+    static final String RESOURCE_REGISTRY_PROPERTIES_EMPTY = absolutePathForResource('/nifi-registry/nifi-registry-empty.properties')
+    static final String RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED = absolutePathForResource('/nifi-registry/nifi-registry-populated-unprotected.properties')
+    static final String RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128 = absolutePathForResource('/nifi-registry/nifi-registry-populated-protected-key-128.properties')
+    static final String RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_256 = absolutePathForResource('/nifi-registry/nifi-registry-populated-protected-key-256.properties')
+    static final String RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_PASSWORD_256 = absolutePathForResource('/nifi-registry/nifi-registry-populated-protected-password-256.properties')
+
+    static final String RESOURCE_REGISTRY_AUTHORIZERS_COMMENTED = absolutePathForResource('/nifi-registry/authorizers-commented.xml')
+    static final String RESOURCE_REGISTRY_AUTHORIZERS_EMPTY = absolutePathForResource('/nifi-registry/authorizers-empty.xml')
+    static final String RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED = absolutePathForResource('/nifi-registry/authorizers-populated-unprotected.xml')
+
+    static final String RESOURCE_REGISTRY_IDENTITY_PROVIDERS_COMMENTED = absolutePathForResource('/nifi-registry/identity-providers-commented.xml')
+    static final String RESOURCE_REGISTRY_IDENTITY_PROVIDERS_EMPTY = absolutePathForResource('/nifi-registry/identity-providers-empty.xml')
+    static final String RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED = absolutePathForResource('/nifi-registry/identity-providers-populated-unprotected.xml')
+
+    static final String[] RESOURCE_REGISTRY_PROPERTIES_SENSITIVE_PROPS = [
+            "nifi.registry.security.keystorePasswd",
+            "nifi.registry.security.keyPasswd",
+            "nifi.registry.security.truststorePasswd",
+            "nifi.registry.dummy.sensitive.property.1",
+            "nifi.registry.dummy.sensitive.property.2"
+    ]
+
+    private static final int RESOURCE_REGISTRY_IDENTITY_PROVIDERS_PASSWORD_LINE_COUNT = 3
+    private static final int RESOURCE_REGISTRY_AUTHORIZERS_PASSWORD_LINE_COUNT = 3
+    private final String PASSWORD_PROP_REGEX = "<property[^>]* name=\".* Password\""
+
+    static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210"
+    static final String KEY_HEX_256 = KEY_HEX_128 * 2
+    static final String KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128
+
+    static final String PASSWORD = "thisIsABadPassword"
+    // From ToolUtilities.deriveKeyFromPassword("thisIsABadPassword")
+    static final String PASSWORD_KEY_HEX_256 = "2C576A9585DB862F5ECBEE5B4FFFCCA14B18D8365968D7081651006507AD2BDE"
+    static final String PASSWORD_KEY_HEX_128 = "2C576A9585DB862F5ECBEE5B4FFFCCA1"
+    static final String PASSWORD_KEY_HEX = isUnlimitedStrengthCryptoAvailable() ? PASSWORD_KEY_HEX_256 : PASSWORD_KEY_HEX_128
+
+    static final String PROTECTION_SCHEME_128 = "aes/gcm/128"
+    static final String PROTECTION_SCHEME_256 = "aes/gcm/256"
+    static final String PROTECTION_SCHEME = isUnlimitedStrengthCryptoAvailable() ? PROTECTION_SCHEME_256 : PROTECTION_SCHEME_128
+
+    private static final String DEFAULT_TMP_DIR = "target/tmp/"
+
+    /**
+     * @return boolean indicating if the current Java Runtime Environment supports unlimited strength crypto functions
+     */
+    static boolean isUnlimitedStrengthCryptoAvailable() {
+        Cipher.getMaxAllowedKeyLength("AES") > 128
+    }
+
+    private static absolutePathForResource(String relativeResourcePath) {
+        return TestUtil.class.getResource(relativeResourcePath).getPath()
+    }
+
+    static File setupTmpDir(String tmpDirPath = DEFAULT_TMP_DIR) {
+        File tmpDir = new File(tmpDirPath)
+        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])
+        tmpDir
+    }
+
+    static void cleanupTmpDir(String tmpDirPath = DEFAULT_TMP_DIR) {
+        File tmpDir = new File(tmpDirPath)
+        tmpDir.delete()
+    }
+
+    static String generateTmpFilePath() {
+        File tmpDir = setupTmpDir()
+        return "${tmpDir.getAbsolutePath()}/${UUID.randomUUID().toString()}.tmp_file"
+    }
+
+    static File generateTmpFile() {
+        File tmpFile = new File(generateTmpFilePath())
+        tmpFile
+    }
+
+    static String copyFileToTempFile(String filePath) {
+        File tmpFile = generateTmpFile()
+        tmpFile.text = new File(filePath).text
+        return tmpFile.getAbsolutePath()
+    }
+
+    /**
+     * OS-agnostic method for setting file permissions. On POSIX-compliant systems, accurately sets the provided permissions. On Windows, sets the corresponding permissions for the file owner only.
+     *
+     * @param file the file to modify
+     * @param permissions the desired permissions
+     */
+    static void setFilePermissions(File file, List<PosixFilePermission> permissions = []) {
+        if (SystemUtils.IS_OS_WINDOWS) {
+            file?.setReadable(permissions.contains(PosixFilePermission.OWNER_READ))
+            file?.setWritable(permissions.contains(PosixFilePermission.OWNER_WRITE))
+            file?.setExecutable(permissions.contains(PosixFilePermission.OWNER_EXECUTE))
+        } else {
+            Files.setPosixFilePermissions(file?.toPath(), permissions as Set)
+        }
+    }
+
+    /**
+     * Make assertions that a properties file is protected correctly given a known starting point.
+     *
+     * @param pathToOriginalUnprotectedProperties - location of the original, plaintext properties file
+     * @param pathToProtectedPropertiesToVerify - location of the protected properties file
+     * @param sensitivePropertiesToVerify - the properties that should be considered sensitive
+     * @param expectedProtectionSchemeToVerify - the expected protection cipher identifier
+     * @return true if all assertion checks pass, otherwise assertion error is thrown
+     */
+    static boolean assertPropertiesAreProtected(
+            String pathToOriginalUnprotectedProperties,
+            String pathToProtectedPropertiesToVerify,
+            String[] sensitivePropertiesToVerify,
+            String expectedProtectionScheme = PROTECTION_SCHEME) {
+
+        Properties unprotectedProperties = new Properties()
+        unprotectedProperties.load(new FileReader(pathToOriginalUnprotectedProperties))
+
+        String[] populatedSensitiveProperties = sensitivePropertiesToVerify.findAll {
+            unprotectedProperties.getProperty(it) != null && unprotectedProperties.getProperty(it).toString().length() > 0
+        }
+        def populatedSensitivePropertiesCount = populatedSensitiveProperties.length
+
+        Properties protectedProperties = new Properties()
+        protectedProperties.load(new FileReader(pathToProtectedPropertiesToVerify))
+
+        // For each populated, sensitive property, one additional "*.protected" property should have been added
+        assert unprotectedProperties.size() + populatedSensitivePropertiesCount == protectedProperties.size()
+
+        // For each populated, sensitive property, ensure its value differs from its original value, and
+        // that no two protected property values match (due to IV, which is unique per-property)
+        Set<String> distinctValues = new HashSet<>()
+        populatedSensitiveProperties.every { key ->
+            def originalValue = unprotectedProperties.getProperty(key)
+            def protectedValue = protectedProperties.getProperty(key)
+            def protectionScheme = protectedProperties.getProperty("${key}.protected")
+
+            assert null != protectedValue
+            assert protectedValue.length() > 0
+            assert originalValue != protectedValue
+            assert expectedProtectionScheme == protectionScheme
+
+            assert !distinctValues.contains(protectedValue)
+            distinctValues.add(protectedValue)
+        }
+
+        return true
+    }
+
+    /**
+     * Make assertions that a NiFi Registry Authorizers XML file is protected correctly given a known starting point.
+     *
+     * @param pathToOriginalUnprotectedXml - location of the original, plaintext XML file
+     * @param pathToProtectedXmlToVerify - location of the protected XML file
+     * @param expectedProtectionScheme - expected scheme/cipher used to encrypt
+     * @param expectedKey - key used to encrypt
+     *
+     * @return true if all assertions pass
+     * @throws AssertionError if any assertion fails
+     */
+    static boolean assertRegistryAuthorizersXmlIsProtected(
+            String pathToOriginalUnprotectedXml,
+            String pathToProtectedXmlToVerify,
+            String expectedProtectionScheme = PROTECTION_SCHEME,
+            String expectedKey = KEY_HEX) {
+
+        return assertXmlIsProtected(
+                pathToOriginalUnprotectedXml,
+                pathToProtectedXmlToVerify,
+                expectedProtectionScheme,
+                expectedKey,
+                { rootNode ->
+                    try {
+                        rootNode.userGroupProvider.find {
+                            it.'class'.text() == NiFiRegistryAuthorizersXmlEncryptor.LDAP_USER_GROUP_PROVIDER_CLASS
+                        }.property.findAll {
+                            it.@name =~ "Password"
+                        }
+                    } catch (Exception ignored) {
+                        null
+                    }
+
+                }
+        )
+    }
+
+    /**
+     * Make assertions that a NiFi Registry Identity Providers XML file is protected correctly given a known starting point.
+     *
+     * @param pathToOriginalUnprotectedXml - location of the original, plaintext XML file
+     * @param pathToProtectedXmlToVerify - location of the protected XML file
+     * @param expectedProtectionScheme - expected scheme/cipher used to encrypt
+     * @param expectedKey - key used to encrypt
+     *
+     * @return true if all assertions pass
+     * @throws AssertionError if any assertion fails
+     */
+    static boolean assertRegistryIdentityProvidersXmlIsProtected(
+            String pathToOriginalUnprotectedXml,
+            String pathToProtectedXmlToVerify,
+            String expectedProtectionScheme = PROTECTION_SCHEME,
+            String expectedKey = KEY_HEX) {
+
+        return assertXmlIsProtected(
+                pathToOriginalUnprotectedXml,
+                pathToProtectedXmlToVerify,
+                expectedProtectionScheme,
+                expectedKey,
+                { rootNode ->
+                    try {
+                        rootNode.provider.find {
+                            it.'class'.text() == NiFiRegistryIdentityProvidersXmlEncryptor.LDAP_PROVIDER_CLASS
+                        }.property.findAll {
+                            it.@name =~ "Password"
+                        }
+                    } catch (Exception ignored) {
+                        null
+                    }
+
+                }
+        )
+    }
+
+    /**
+     * Make assertions that an XML file is protected correctly given a known starting point.
+     *
+     * @param pathToOriginalUnprotectedXml - location of the original, plaintext XML file
+     * @param pathToProtectedXmlToVerify - location of the protected XML file
+     * @param expectedProtectionScheme - expected scheme/cipher used to encrypt
+     * @param expectedKey - key used to encrypt
+     * @param callbackToGetNodesToVerify - closure that returns GPathResult[] of all sensitive nodes that
+     *                                     should be protected given a GPathResult for the root of the XML document
+     *
+     * @return true if all assertions pass
+     * @throws AssertionError if any assertion fails
+     */
+    static boolean assertXmlIsProtected(
+            String pathToOriginalUnprotectedXml,
+            String pathToProtectedXmlToVerify,
+            String expectedProtectionScheme = PROTECTION_SCHEME,
+            String expectedKey = KEY_HEX,
+            callbackToGetNodesToVerify) {
+
+        String originalUnprotectedXml = new File(pathToOriginalUnprotectedXml).text
+        String protectedXml = new File(pathToProtectedXmlToVerify).text
+        def originalDoc = new XmlParser().parseText(originalUnprotectedXml)
+        def protectedDoc = new XmlParser().parseText(protectedXml)
+
+        def sensitiveProperties = callbackToGetNodesToVerify(originalDoc)
+        assert sensitiveProperties && sensitiveProperties.size > 0  // necessary as so many key assertions are based on at least one sensitive prop
+        def populatedSensitiveProperties = sensitiveProperties.findAll { node ->
+            node.text()
+        }
+        def plaintextValues = populatedSensitiveProperties.collect {
+            it.text()
+        }
+
+        if (populatedSensitiveProperties.size() == 0) {
+            return assertFilesAreEqual(pathToOriginalUnprotectedXml, pathToProtectedXmlToVerify)
+        }
+
+        def protectedSensitiveProperties = callbackToGetNodesToVerify(protectedDoc).findAll { node ->
+            node.@encryption != "none" && node.@encryption != "" }
+
+        assert populatedSensitiveProperties.size() == protectedSensitiveProperties.size()
+
+        AESSensitivePropertyProvider spp = new AESSensitivePropertyProvider(expectedKey)
+
+        protectedSensitiveProperties.each {
+            String value = it.text()
+            String propertyValue = value
+            assert it.@encryption == expectedProtectionScheme
+            assert !plaintextValues.contains(propertyValue)
+            assert plaintextValues.contains(spp.unprotect(propertyValue))
+        }
+
+        return true
+    }
+
+    /**
+     * Asserts the contents of files are equal, ignoring blank lines and starting / trailing whitespace
+     *
+     * @param pathToExpected - path to file with the expected content
+     * @param pathToActual - path to file with the actual content
+     * @return true if assertions pass
+     */
+    static boolean assertFilesAreEqual(String pathToExpected, String pathToActual) {
+        List<String> expectedLines = new File(pathToExpected).readLines().findAll{
+            it.trim().length() > 0
+        }.collect{ it.trim() }
+        List<String> actualLines = new File(pathToActual).readLines().findAll{
+            it.trim().length() > 0
+        }.collect{ it.trim() }
+
+        return assertLinesAreEqual(expectedLines, actualLines)
+    }
+
+    /**
+     * Asserts the contents of a bootstrap.conf file match that of an an expected bootstrap.conf.
+     *
+     * @param pathToExpectedBootstrap
+     * @param pathToActualBootstrap
+     * @param includeComments - if false, comment lines in the bootstrap.conf files will be ignored
+     * @return true if assertions pass
+     */
+    static boolean assertBootstrapFilesAreEqual(String pathToExpectedBootstrap, String pathToActualBootstrap, boolean includeComments) {
+        return assertConfOrPropertiesFilesAreEqual(pathToExpectedBootstrap, pathToActualBootstrap, includeComments)
+    }
+
+    /**
+     * Asserts the contents of a properties file match that of an an expected properties file.
+     *
+     * @param pathToExpectedProperties
+     * @param pathToActualProperties
+     * @param includeComments - if false, comment lines in the properties files will be ignored
+     * @return true if assertions pass
+     */
+    static boolean assertPropertiesFilesAreEqual(String pathToExpectedProperties, String pathToActualProperties, boolean includeComments) {
+        return assertConfOrPropertiesFilesAreEqual(pathToExpectedProperties, pathToActualProperties, includeComments)
+    }
+
+    private static boolean assertConfOrPropertiesFilesAreEqual(String expected, String actual, boolean includeComments) {
+        List<String> expectedLines = new File(expected).readLines().findAll{
+            (it.trim().length() > 0 && (includeComments || !it.startsWith("#")))
+        }.collect{ it.trim() }
+        List<String> actualLines = new File(actual).readLines().findAll{
+            (it.trim().length() > 0 && (includeComments || !it.startsWith("#")))
+        }.collect{ it.trim() }
+
+        return assertLinesAreEqual(expectedLines, actualLines)
+    }
+
+    private static boolean assertLinesAreEqual(List<String> expectedLines, List<String> actualLines) {
+
+        assert actualLines != null
+        assert actualLines.size() == expectedLines.size()
+        assert actualLines == expectedLines
+
+        return true
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/util/BootstrapUtilSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/util/BootstrapUtilSpec.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/util/BootstrapUtilSpec.groovy
new file mode 100644
index 0000000..1278a58
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/util/BootstrapUtilSpec.groovy
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.toolkit.encryptconfig.util
+
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+import spock.lang.Specification
+
+import static org.apache.nifi.toolkit.encryptconfig.TestUtil.*
+
+class BootstrapUtilSpec extends Specification {
+    private static final Logger logger = LoggerFactory.getLogger(BootstrapUtilSpec.class)
+
+    // runs before every feature method
+    def setup() {}
+
+    // runs after every feature method
+    def cleanup() {}
+
+    // runs before the first feature method
+    def setupSpec() {
+        setupTmpDir()
+    }
+
+    // runs after the last feature method
+    def cleanupSpec() {
+        cleanupTmpDir()
+    }
+
+    def "test extractKeyFromBootstrapFile with Registry bootstrap.conf"() {
+
+        setup:
+        def bootstrapKeyProperty = BootstrapUtil.REGISTRY_BOOTSTRAP_KEY_PROPERTY
+
+
+        when: "bootstrap.conf has no key property"
+        def actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY, bootstrapKeyProperty)
+
+        then: "null is returned"
+        actualKeyHex == null
+
+
+        when: "bootstrap.conf has an empty key property"
+        actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(RESOURCE_REGISTRY_BOOTSTRAP_EMPTY_KEY, bootstrapKeyProperty)
+
+        then: "null is returned"
+        actualKeyHex == null
+
+
+        when: "bootstrap.conf has a populated key property"
+        actualKeyHex = BootstrapUtil.extractKeyFromBootstrapFile(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128, bootstrapKeyProperty)
+
+        then: "key is returned"
+        actualKeyHex == KEY_HEX_128
+
+
+        when: "bootstrap.conf file does not exist"
+        BootstrapUtil.extractKeyFromBootstrapFile("__file_does_not_exist__", bootstrapKeyProperty)
+
+        then: "expect an IOException"
+        thrown IOException
+
+    }
+
+    def "test writeKeyToBootstrapFile with Registry bootstrap.conf"() {
+
+        setup:
+        def bootstrapKeyProperty = BootstrapUtil.REGISTRY_BOOTSTRAP_KEY_PROPERTY
+        def outFile1 = generateTmpFilePath()
+        def outFile2 = generateTmpFilePath()
+        def outFile3 = generateTmpFilePath()
+        def expected = RESOURCE_REGISTRY_BOOTSTRAP_KEY_128
+
+
+        when: "input is default bootstrap.conf"
+        BootstrapUtil.writeKeyToBootstrapFile(KEY_HEX_128, bootstrapKeyProperty, outFile1, RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+
+        then: "output file content matches populated bootstrap file"
+        assertBootstrapFilesAreEqual(expected, outFile1, true)
+        and: "key is readable from output file"
+        BootstrapUtil.extractKeyFromBootstrapFile(outFile1, bootstrapKeyProperty) == KEY_HEX_128
+
+
+        when: "input bootstrap.conf has no key property"
+        BootstrapUtil.writeKeyToBootstrapFile(KEY_HEX_128, bootstrapKeyProperty, outFile2, RESOURCE_REGISTRY_BOOTSTRAP_NO_KEY)
+
+        then: "output file content matches pre-populated bootstrap file"
+        assertBootstrapFilesAreEqual(expected, outFile2, true)
+
+
+        when: "input bootstrap.conf has existing, different master key"
+        BootstrapUtil.writeKeyToBootstrapFile(KEY_HEX_128, bootstrapKeyProperty, outFile3, RESOURCE_REGISTRY_BOOTSTRAP_KEY_FROM_PASSWORD_128)
+
+        then: "output file content matches pre-populated bootstrap file"
+        assertBootstrapFilesAreEqual(expected, outFile3, true)
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-commented.xml
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-commented.xml b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-commented.xml
new file mode 100644
index 0000000..af0c531
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-commented.xml
@@ -0,0 +1,242 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<!--
+    This file lists the userGroupProviders, accessPolicyProviders, and authorizers to use when running securely. In order
+    to use a specific authorizer it must be configured here and its identifier must be specified in the nifi-registry.properties file.
+    If the authorizer is a managedAuthorizer, it may need to be configured with an accessPolicyProvider and an userGroupProvider.
+    This file allows for configuration of them, but they must be configured in order:
+
+    ...
+    all userGroupProviders
+    all accessPolicyProviders
+    all Authorizers
+    ...
+-->
+<authorizers>
+
+    <!--
+        The FileUserGroupProvider will provide support for managing users and groups which is backed by a file
+        on the local file system.
+
+        - Users File - The file where the FileUserGroupProvider will store users and groups.
+
+        - Initial User Identity [unique key] - The identity of a users and systems to seed the Users File. The name of
+            each property must be unique, for example: "Initial User Identity A", "Initial User Identity B",
+            "Initial User Identity C" or "Initial User Identity 1", "Initial User Identity 2", "Initial User Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the user identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate).
+    -->
+    <userGroupProvider>
+        <identifier>file-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.file.FileUserGroupProvider</class>
+        <property name="Users File">./conf/users.xml</property>
+        <property name="Initial User Identity 1"><!--CN=abc, OU=xyz--></property>
+    </userGroupProvider>
+
+    <!--
+        The LdapUserGroupProvider will retrieve users and groups from an LDAP server. The users and groups
+        are not configurable.
+
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, LDAPS, or START_TLS.
+
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using LDAPS or START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using LDAPS or START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using LDAPS or START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully
+            before the target context is closed. Defaults to false.
+
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+
+        'Url' - Space-separated list of URLs of the LDAP servers (i.e. ldap://<hostname>:<port>).
+        'Page Size' - Sets the page size when retrieving users and groups. If not specified, no paging is performed.
+        'Sync Interval' - Duration of time between syncing users and groups. (i.e. 30 mins).
+
+        'User Search Base' - Base DN for searching for users (i.e. ou=users,o=nifi). Required to search users.
+        'User Object Class' - Object class for identifying users (i.e. person). Required if searching users.
+        'User Search Scope' - Search scope for searching users (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching users.
+        'User Search Filter' - Filter for searching for users against the 'User Search Base' (i.e. (memberof=cn=team1,ou=groups,o=nifi) ). Optional.
+        'User Identity Attribute' - Attribute to use to extract user identity (i.e. cn). Optional. If not set, the entire DN is used.
+        'User Group Name Attribute' - Attribute to use to define group membership (i.e. memberof). Optional. If not set
+            group membership will not be calculated through the users. Will rely on group membership being defined
+            through 'Group Member Attribute' if set.
+
+        'Group Search Base' - Base DN for searching for groups (i.e. ou=groups,o=nifi). Required to search groups.
+        'Group Object Class' - Object class for identifying groups (i.e. groupOfNames). Required if searching groups.
+        'Group Search Scope' - Search scope for searching groups (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching groups.
+        'Group Search Filter' - Filter for searching for groups against the 'Group Search Base'. Optional.
+        'Group Name Attribute' - Attribute to use to extract group name (i.e. cn). Optional. If not set, the entire DN is used.
+        'Group Member Attribute' - Attribute to use to define group membership (i.e. member). Optional. If not set
+            group membership will not be calculated through the groups. Will rely on group member being defined
+            through 'User Group Name Attribute' if set.
+
+        NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the user identities.
+            Group names are not mapped.
+    -->
+    <!-- To enable the ldap-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>ldap-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.ldap.tenants.LdapUserGroupProvider</class>
+        <property name="Authentication Strategy">START_TLS</property>
+
+        <property name="Manager DN"></property>
+        <property name="Manager Password"></property>
+
+        <property name="TLS - Keystore"></property>
+        <property name="TLS - Keystore Password"></property>
+        <property name="TLS - Keystore Type"></property>
+        <property name="TLS - Truststore"></property>
+        <property name="TLS - Truststore Password"></property>
+        <property name="TLS - Truststore Type"></property>
+        <property name="TLS - Client Auth"></property>
+        <property name="TLS - Protocol"></property>
+        <property name="TLS - Shutdown Gracefully"></property>
+
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url"></property>
+        <property name="Page Size"></property>
+        <property name="Sync Interval">30 mins</property>
+
+        <property name="User Search Base"></property>
+        <property name="User Object Class">person</property>
+        <property name="User Search Scope">ONE_LEVEL</property>
+        <property name="User Search Filter"></property>
+        <property name="User Identity Attribute"></property>
+        <property name="User Group Name Attribute"></property>
+
+        <property name="Group Search Base"></property>
+        <property name="Group Object Class">group</property>
+        <property name="Group Search Scope">ONE_LEVEL</property>
+        <property name="Group Search Filter"></property>
+        <property name="Group Name Attribute"></property>
+        <property name="Group Member Attribute"></property>
+    </userGroupProvider>
+    To enable the ldap-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The CompositeUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties are not applied in this implementation. This
+            behavior would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.CompositeUserGroupProvider</class>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The CompositeConfigurableUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+        Additionally, a single configurable user group provider is required. Users from the configurable user group provider
+        are configurable, however users loaded from one of the User Group Provider [unique key] will not be.
+
+        - Configurable User Group Provider - A configurable user group provider.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties are not applied in this implementation. This
+            behavior would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-configurable-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-configurable-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.CompositeConfigurableUserGroupProvider</class>
+        <property name="Configurable User Group Provider">file-user-group-provider</property>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-configurable-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The FileAccessPolicyProvider will provide support for managing access policies which is backed by a file
+        on the local file system.
+
+        - User Group Provider - The identifier for an User Group Provider defined above that will be used to access
+            users and groups for use in the managed access policies.
+
+        - Authorizations File - The file where the FileAccessPolicyProvider will store policies.
+
+        - Initial Admin Identity - The identity of an initial admin user that will be granted access to the UI and
+            given the ability to create additional users, groups, and policies. The value of this property could be
+            a DN when using certificates or LDAP. This property will only be used when there
+            are no other policies defined.
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the initial admin identity,
+            so the value should be the unmapped identity. This identity must be found in the configured User Group Provider.
+
+        - NiFi Identity [unique key] - The identity of a NiFi node that will have access to this NiFi Registry and will be able
+            to act as a proxy on behalf of a NiFi Registry end user. A property should be created for the identity of every NiFi
+            node that needs to access this NiFi Registry. The name of each property must be unique, for example for three
+            NiFi clients:
+            "NiFi Identity A", "NiFi Identity B", "NiFi Identity C" or "NiFi Identity 1", "NiFi Identity 2", "NiFi Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the nifi identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate). This identity must be found
+            in the configured User Group Provider.
+    -->
+    <accessPolicyProvider>
+        <identifier>file-access-policy-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.file.FileAccessPolicyProvider</class>
+        <property name="User Group Provider">file-user-group-provider</property>
+        <property name="Authorizations File">./conf/authorizations.xml</property>
+        <property name="Initial Admin Identity"><!-- CN=abc, OU=xyz --></property>
+
+        <!--<property name="NiFi Identity 1"></property>-->
+    </accessPolicyProvider>
+
+    <!--
+        The StandardManagedAuthorizer. This authorizer implementation must be configured with the
+        Access Policy Provider which it will use to access and manage users, groups, and policies.
+        These users, groups, and policies will be used to make all access decisions during authorization
+        requests.
+
+        - Access Policy Provider - The identifier for an Access Policy Provider defined above.
+    -->
+    <authorizer>
+        <identifier>managed-authorizer</identifier>
+        <class>org.apache.nifi.registry.security.authorization.StandardManagedAuthorizer</class>
+        <property name="Access Policy Provider">file-access-policy-provider</property>
+    </authorizer>
+
+</authorizers>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-empty.xml
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-empty.xml b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-empty.xml
new file mode 100644
index 0000000..d70d907
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-empty.xml
@@ -0,0 +1,240 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<!--
+    This file lists the userGroupProviders, accessPolicyProviders, and authorizers to use when running securely. In order
+    to use a specific authorizer it must be configured here and its identifier must be specified in the nifi-registry.properties file.
+    If the authorizer is a managedAuthorizer, it may need to be configured with an accessPolicyProvider and an userGroupProvider.
+    This file allows for configuration of them, but they must be configured in order:
+
+    ...
+    all userGroupProviders
+    all accessPolicyProviders
+    all Authorizers
+    ...
+-->
+<authorizers>
+
+    <!--
+        The FileUserGroupProvider will provide support for managing users and groups which is backed by a file
+        on the local file system.
+
+        - Users File - The file where the FileUserGroupProvider will store users and groups.
+
+        - Initial User Identity [unique key] - The identity of a users and systems to seed the Users File. The name of
+            each property must be unique, for example: "Initial User Identity A", "Initial User Identity B",
+            "Initial User Identity C" or "Initial User Identity 1", "Initial User Identity 2", "Initial User Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the user identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate).
+    -->
+    <userGroupProvider>
+        <identifier>file-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.file.FileUserGroupProvider</class>
+        <property name="Users File">./conf/users.xml</property>
+        <property name="Initial User Identity 1"><!--CN=abc, OU=xyz--></property>
+    </userGroupProvider>
+
+    <!--
+        The LdapUserGroupProvider will retrieve users and groups from an LDAP server. The users and groups
+        are not configurable.
+
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, LDAPS, or START_TLS.
+
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using LDAPS or START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using LDAPS or START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using LDAPS or START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully
+            before the target context is closed. Defaults to false.
+
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+
+        'Url' - Space-separated list of URLs of the LDAP servers (i.e. ldap://<hostname>:<port>).
+        'Page Size' - Sets the page size when retrieving users and groups. If not specified, no paging is performed.
+        'Sync Interval' - Duration of time between syncing users and groups. (i.e. 30 mins).
+
+        'User Search Base' - Base DN for searching for users (i.e. ou=users,o=nifi). Required to search users.
+        'User Object Class' - Object class for identifying users (i.e. person). Required if searching users.
+        'User Search Scope' - Search scope for searching users (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching users.
+        'User Search Filter' - Filter for searching for users against the 'User Search Base' (i.e. (memberof=cn=team1,ou=groups,o=nifi) ). Optional.
+        'User Identity Attribute' - Attribute to use to extract user identity (i.e. cn). Optional. If not set, the entire DN is used.
+        'User Group Name Attribute' - Attribute to use to define group membership (i.e. memberof). Optional. If not set
+            group membership will not be calculated through the users. Will rely on group membership being defined
+            through 'Group Member Attribute' if set.
+
+        'Group Search Base' - Base DN for searching for groups (i.e. ou=groups,o=nifi). Required to search groups.
+        'Group Object Class' - Object class for identifying groups (i.e. groupOfNames). Required if searching groups.
+        'Group Search Scope' - Search scope for searching groups (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching groups.
+        'Group Search Filter' - Filter for searching for groups against the 'Group Search Base'. Optional.
+        'Group Name Attribute' - Attribute to use to extract group name (i.e. cn). Optional. If not set, the entire DN is used.
+        'Group Member Attribute' - Attribute to use to define group membership (i.e. member). Optional. If not set
+            group membership will not be calculated through the groups. Will rely on group member being defined
+            through 'User Group Name Attribute' if set.
+
+        NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the user identities.
+            Group names are not mapped.
+    -->
+    <userGroupProvider>
+        <identifier>ldap-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.ldap.tenants.LdapUserGroupProvider</class>
+        <property name="Authentication Strategy">START_TLS</property>
+
+        <property name="Manager DN"></property>
+        <property name="Manager Password"></property>
+
+        <property name="TLS - Keystore"></property>
+        <property name="TLS - Keystore Password"></property>
+        <property name="TLS - Keystore Type"></property>
+        <property name="TLS - Truststore"></property>
+        <property name="TLS - Truststore Password"></property>
+        <property name="TLS - Truststore Type"></property>
+        <property name="TLS - Client Auth"></property>
+        <property name="TLS - Protocol"></property>
+        <property name="TLS - Shutdown Gracefully"></property>
+
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url"></property>
+        <property name="Page Size"></property>
+        <property name="Sync Interval">30 mins</property>
+
+        <property name="User Search Base"></property>
+        <property name="User Object Class">person</property>
+        <property name="User Search Scope">ONE_LEVEL</property>
+        <property name="User Search Filter"></property>
+        <property name="User Identity Attribute"></property>
+        <property name="User Group Name Attribute"></property>
+
+        <property name="Group Search Base"></property>
+        <property name="Group Object Class">group</property>
+        <property name="Group Search Scope">ONE_LEVEL</property>
+        <property name="Group Search Filter"></property>
+        <property name="Group Name Attribute"></property>
+        <property name="Group Member Attribute"></property>
+    </userGroupProvider>
+
+    <!--
+        The CompositeUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties are not applied in this implementation. This
+            behavior would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.CompositeUserGroupProvider</class>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The CompositeConfigurableUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+        Additionally, a single configurable user group provider is required. Users from the configurable user group provider
+        are configurable, however users loaded from one of the User Group Provider [unique key] will not be.
+
+        - Configurable User Group Provider - A configurable user group provider.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties are not applied in this implementation. This
+            behavior would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-configurable-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-configurable-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.CompositeConfigurableUserGroupProvider</class>
+        <property name="Configurable User Group Provider">file-user-group-provider</property>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-configurable-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The FileAccessPolicyProvider will provide support for managing access policies which is backed by a file
+        on the local file system.
+
+        - User Group Provider - The identifier for an User Group Provider defined above that will be used to access
+            users and groups for use in the managed access policies.
+
+        - Authorizations File - The file where the FileAccessPolicyProvider will store policies.
+
+        - Initial Admin Identity - The identity of an initial admin user that will be granted access to the UI and
+            given the ability to create additional users, groups, and policies. The value of this property could be
+            a DN when using certificates or LDAP. This property will only be used when there
+            are no other policies defined.
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the initial admin identity,
+            so the value should be the unmapped identity. This identity must be found in the configured User Group Provider.
+
+        - NiFi Identity [unique key] - The identity of a NiFi node that will have access to this NiFi Registry and will be able
+            to act as a proxy on behalf of a NiFi Registry end user. A property should be created for the identity of every NiFi
+            node that needs to access this NiFi Registry. The name of each property must be unique, for example for three
+            NiFi clients:
+            "NiFi Identity A", "NiFi Identity B", "NiFi Identity C" or "NiFi Identity 1", "NiFi Identity 2", "NiFi Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the nifi identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate). This identity must be found
+            in the configured User Group Provider.
+    -->
+    <accessPolicyProvider>
+        <identifier>file-access-policy-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.file.FileAccessPolicyProvider</class>
+        <property name="User Group Provider">file-user-group-provider</property>
+        <property name="Authorizations File">./conf/authorizations.xml</property>
+        <property name="Initial Admin Identity"><!-- CN=abc, OU=xyz --></property>
+
+        <!--<property name="NiFi Identity 1"></property>-->
+    </accessPolicyProvider>
+
+    <!--
+        The StandardManagedAuthorizer. This authorizer implementation must be configured with the
+        Access Policy Provider which it will use to access and manage users, groups, and policies.
+        These users, groups, and policies will be used to make all access decisions during authorization
+        requests.
+
+        - Access Policy Provider - The identifier for an Access Policy Provider defined above.
+    -->
+    <authorizer>
+        <identifier>managed-authorizer</identifier>
+        <class>org.apache.nifi.registry.security.authorization.StandardManagedAuthorizer</class>
+        <property name="Access Policy Provider">file-access-policy-provider</property>
+    </authorizer>
+
+</authorizers>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-populated-unprotected.xml
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-populated-unprotected.xml b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-populated-unprotected.xml
new file mode 100644
index 0000000..258ce28
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/authorizers-populated-unprotected.xml
@@ -0,0 +1,246 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~     http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+<!--
+    This file lists the userGroupProviders, accessPolicyProviders, and authorizers to use when running securely. In order
+    to use a specific authorizer it must be configured here and its identifier must be specified in the nifi-registry.properties file.
+    If the authorizer is a managedAuthorizer, it may need to be configured with an accessPolicyProvider and an userGroupProvider.
+    This file allows for configuration of them, but they must be configured in order:
+
+    ...
+    all userGroupProviders
+    all accessPolicyProviders
+    all Authorizers
+    ...
+-->
+<authorizers>
+
+    <!--
+        The FileUserGroupProvider will provide support for managing users and groups which is backed by a file
+        on the local file system.
+
+        - Users File - The file where the FileUserGroupProvider will store users and groups.
+
+        - Initial User Identity [unique key] - The identity of a users and systems to seed the Users File. The name of
+            each property must be unique, for example: "Initial User Identity A", "Initial User Identity B",
+            "Initial User Identity C" or "Initial User Identity 1", "Initial User Identity 2", "Initial User Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the user identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate).
+    -->
+    <!--
+    <userGroupProvider>
+        <identifier>file-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.authorization.file.FileUserGroupProvider</class>
+        <property name="Users File">./conf/users.xml</property>
+        <property name="Initial User Identity 1">CN=kdoran, OU=NIFI</property>
+    </userGroupProvider>
+    -->
+
+    <!--
+        The LdapUserGroupProvider will retrieve users and groups from an LDAP server. The users and groups
+        are not configurable.
+
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, LDAPS, or START_TLS.
+
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using LDAPS or START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using LDAPS or START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using LDAPS or START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully
+            before the target context is closed. Defaults to false.
+
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+
+        'Url' - Space-separated list of URLs of the LDAP servers (i.e. ldap://<hostname>:<port>).
+        'Page Size' - Sets the page size when retrieving users and groups. If not specified, no paging is performed.
+        'Sync Interval' - Duration of time between syncing users and groups. (i.e. 30 mins).
+
+        'User Search Base' - Base DN for searching for users (i.e. ou=users,o=nifi). Required to search users.
+        'User Object Class' - Object class for identifying users (i.e. person). Required if searching users.
+        'User Search Scope' - Search scope for searching users (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching users.
+        'User Search Filter' - Filter for searching for users against the 'User Search Base' (i.e. (memberof=cn=team1,ou=groups,o=nifi) ). Optional.
+        'User Identity Attribute' - Attribute to use to extract user identity (i.e. cn). Optional. If not set, the entire DN is used.
+        'User Group Name Attribute' - Attribute to use to define group membership (i.e. memberof). Optional. If not set
+            group membership will not be calculated through the users. Will rely on group membership being defined
+            through 'Group Member Attribute' if set.
+
+        'Group Search Base' - Base DN for searching for groups (i.e. ou=groups,o=nifi). Required to search groups.
+        'Group Object Class' - Object class for identifying groups (i.e. groupOfNames). Required if searching groups.
+        'Group Search Scope' - Search scope for searching groups (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching groups.
+        'Group Search Filter' - Filter for searching for groups against the 'Group Search Base'. Optional.
+        'Group Name Attribute' - Attribute to use to extract group name (i.e. cn). Optional. If not set, the entire DN is used.
+        'Group Member Attribute' - Attribute to use to define group membership (i.e. member). Optional. If not set
+            group membership will not be calculated through the groups. Will rely on group member being defined
+            through 'User Group Name Attribute' if set.
+
+        NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the user identities.
+            Group names are not mapped.
+    -->
+    <userGroupProvider>
+        <identifier>ldap-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.ldap.tenants.LdapUserGroupProvider</class>
+        <property name="Authentication Strategy">START_TLS</property>
+
+        <property name="Manager DN">someuser</property>
+        <property name="Manager Password">thisIsABadPassword</property>
+
+        <property name="TLS - Keystore"></property>
+        <property name="TLS - Keystore Password">thisIsABadPassword</property>
+        <property name="TLS - Keystore Type"></property>
+        <property name="TLS - Truststore"></property>
+        <property name="TLS - Truststore Password">thisIsABadPassword</property>
+        <property name="TLS - Truststore Type"></property>
+        <property name="TLS - Client Auth"></property>
+        <property name="TLS - Protocol"></property>
+        <property name="TLS - Shutdown Gracefully"></property>
+
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url"></property>
+        <property name="Page Size"></property>
+        <property name="Sync Interval">30 mins</property>
+
+        <property name="User Search Base"></property>
+        <property name="User Object Class">person</property>
+        <property name="User Search Scope">ONE_LEVEL</property>
+        <property name="User Search Filter"></property>
+        <property name="User Identity Attribute"></property>
+        <property name="User Group Name Attribute"></property>
+        <property name="User Group Name Attribute - Referenced Group Attribute"></property>
+
+        <property name="Group Search Base"></property>
+        <property name="Group Object Class">group</property>
+        <property name="Group Search Scope">ONE_LEVEL</property>
+        <property name="Group Search Filter"></property>
+        <property name="Group Name Attribute"></property>
+        <property name="Group Member Attribute"></property>
+        <property name="Group Member Attribute - Referenced User Attribute"></property>
+    </userGroupProvider>
+
+    <!--
+        The CompositeUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties are not applied in this implementation. This behavior
+            would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.CompositeUserGroupProvider</class>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The CompositeConfigurableUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+        Additionally, a single configurable user group provider is required. Users from the configurable user group provider
+        are configurable, however users loaded from one of the User Group Provider [unique key] will not be.
+
+        - Configurable User Group Provider - A configurable user group provider.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties are not applied in this implementation. This behavior
+            would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-configurable-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-configurable-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.CompositeConfigurableUserGroupProvider</class>
+        <property name="Configurable User Group Provider">file-user-group-provider</property>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-configurable-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The FileAccessPolicyProvider will provide support for managing access policies which is backed by a file
+        on the local file system.
+
+        - User Group Provider - The identifier for an User Group Provider defined above that will be used to access
+            users and groups for use in the managed access policies.
+
+        - Authorizations File - The file where the FileAccessPolicyProvider will store policies.
+
+        - Initial Admin Identity - The identity of an initial admin user that will be granted access to the UI and
+            given the ability to create additional users, groups, and policies. The value of this property could be
+            a DN when using certificates or LDAP. This property will only be used when there
+            are no other policies defined.
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the initial admin identity,
+            so the value should be the unmapped identity. This identity must be found in the configured User Group Provider.
+
+        - NiFi Identity [unique key] - The identity of a NiFi node that will have access to this NiFi Registry and will be able
+            to act as a proxy on behalf of a NiFi Registry end user. A property should be created for the identity of every NiFi
+            node that needs to access this NiFi Registry. The name of each property must be unique, for example for three
+            NiFi clients:
+            "NiFi Identity A", "NiFi Identity B", "NiFi Identity C" or "NiFi Identity 1", "NiFi Identity 2", "NiFi Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi-registry.properties will also be applied to the nifi identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate). This identity must be found
+            in the configured User Group Provider.
+    -->
+    <accessPolicyProvider>
+        <identifier>file-access-policy-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.file.FileAccessPolicyProvider</class>
+        <property name="User Group Provider">ldap-user-group-provider</property>
+        <property name="Authorizations File">./conf/authorizations.xml</property>
+        <property name="Initial Admin Identity">nobel</property>
+
+        <!--<property name="NiFi Identity 1"></property>-->
+    </accessPolicyProvider>
+
+    <!--
+        The StandardManagedAuthorizer. This authorizer implementation must be configured with the
+        Access Policy Provider which it will use to access and manage users, groups, and policies.
+        These users, groups, and policies will be used to make all access decisions during authorization
+        requests.
+
+        - Access Policy Provider - The identifier for an Access Policy Provider defined above.
+    -->
+    <authorizer>
+        <identifier>managed-authorizer</identifier>
+        <class>org.apache.nifi.registry.security.authorization.StandardManagedAuthorizer</class>
+        <property name="Access Policy Provider">file-access-policy-provider</property>
+    </authorizer>
+
+</authorizers>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_default.conf
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_default.conf b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_default.conf
new file mode 100644
index 0000000..637eb64
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_default.conf
@@ -0,0 +1,48 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Java command to use when running nifi-registry
+java=java
+
+# Username to use when running nifi-registry. This value will be ignored on Windows.
+run.as=
+
+# Configure where nifi-registry's lib and conf directories live
+lib.dir=./lib
+conf.dir=./conf
+
+# How long to wait after telling nifi-registry to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# Disable JSR 199 so that we can use JSP's without running a JDK
+java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
+
+# JVM memory settings
+java.arg.2=-Xms512m
+java.arg.3=-Xmx512m
+
+# Enable Remote Debugging
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+
+java.arg.4=-Djava.net.preferIPv4Stack=true
+
+# allowRestrictedHeaders is required for Cluster/Node communications to work properly
+java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
+java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
+
+# Master key in hexadecimal format for encrypted sensitive configuration values
+nifi.registry.bootstrap.sensitive.key=
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_empty_master_key.conf
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_empty_master_key.conf b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_empty_master_key.conf
new file mode 100644
index 0000000..637eb64
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_empty_master_key.conf
@@ -0,0 +1,48 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Java command to use when running nifi-registry
+java=java
+
+# Username to use when running nifi-registry. This value will be ignored on Windows.
+run.as=
+
+# Configure where nifi-registry's lib and conf directories live
+lib.dir=./lib
+conf.dir=./conf
+
+# How long to wait after telling nifi-registry to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# Disable JSR 199 so that we can use JSP's without running a JDK
+java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
+
+# JVM memory settings
+java.arg.2=-Xms512m
+java.arg.3=-Xmx512m
+
+# Enable Remote Debugging
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+
+java.arg.4=-Djava.net.preferIPv4Stack=true
+
+# allowRestrictedHeaders is required for Cluster/Node communications to work properly
+java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
+java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
+
+# Master key in hexadecimal format for encrypted sensitive configuration values
+nifi.registry.bootstrap.sensitive.key=
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_128.conf
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_128.conf b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_128.conf
new file mode 100644
index 0000000..6e128bd
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_128.conf
@@ -0,0 +1,48 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Java command to use when running nifi-registry
+java=java
+
+# Username to use when running nifi-registry. This value will be ignored on Windows.
+run.as=
+
+# Configure where nifi-registry's lib and conf directories live
+lib.dir=./lib
+conf.dir=./conf
+
+# How long to wait after telling nifi-registry to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# Disable JSR 199 so that we can use JSP's without running a JDK
+java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
+
+# JVM memory settings
+java.arg.2=-Xms512m
+java.arg.3=-Xmx512m
+
+# Enable Remote Debugging
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+
+java.arg.4=-Djava.net.preferIPv4Stack=true
+
+# allowRestrictedHeaders is required for Cluster/Node communications to work properly
+java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
+java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
+
+# Master key in hexadecimal format for encrypted sensitive configuration values
+nifi.registry.bootstrap.sensitive.key=0123456789ABCDEFFEDCBA9876543210
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_from_password_128.conf
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_from_password_128.conf b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_from_password_128.conf
new file mode 100644
index 0000000..16788a0
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_with_master_key_from_password_128.conf
@@ -0,0 +1,48 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Java command to use when running nifi-registry
+java=java
+
+# Username to use when running nifi-registry. This value will be ignored on Windows.
+run.as=
+
+# Configure where nifi-registry's lib and conf directories live
+lib.dir=./lib
+conf.dir=./conf
+
+# How long to wait after telling nifi-registry to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# Disable JSR 199 so that we can use JSP's without running a JDK
+java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
+
+# JVM memory settings
+java.arg.2=-Xms512m
+java.arg.3=-Xmx512m
+
+# Enable Remote Debugging
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+
+java.arg.4=-Djava.net.preferIPv4Stack=true
+
+# allowRestrictedHeaders is required for Cluster/Node communications to work properly
+java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
+java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol
+
+# Master key in hexadecimal format for encrypted sensitive configuration values
+nifi.registry.bootstrap.sensitive.key=2C576A9585DB862F5ECBEE5B4FFFCCA1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_without_master_key.conf
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_without_master_key.conf b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_without_master_key.conf
new file mode 100644
index 0000000..7abf30c
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/resources/nifi-registry/bootstrap_without_master_key.conf
@@ -0,0 +1,45 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Java command to use when running nifi-registry
+java=java
+
+# Username to use when running nifi-registry. This value will be ignored on Windows.
+run.as=
+
+# Configure where nifi-registry's lib and conf directories live
+lib.dir=./lib
+conf.dir=./conf
+
+# How long to wait after telling nifi-registry to shutdown before explicitly killing the Process
+graceful.shutdown.seconds=20
+
+# Disable JSR 199 so that we can use JSP's without running a JDK
+java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
+
+# JVM memory settings
+java.arg.2=-Xms512m
+java.arg.3=-Xmx512m
+
+# Enable Remote Debugging
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+
+java.arg.4=-Djava.net.preferIPv4Stack=true
+
+# allowRestrictedHeaders is required for Cluster/Node communications to work properly
+java.arg.5=-Dsun.net.http.allowRestrictedHeaders=true
+java.arg.6=-Djava.protocol.handler.pkgs=sun.net.www.protocol