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:06 UTC

[3/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/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy
new file mode 100644
index 0000000..102ad9e
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryAuthorizersXmlEncryptor.groovy
@@ -0,0 +1,103 @@
+/*
+ * 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 groovy.xml.XmlUtil
+import org.apache.nifi.properties.SensitivePropertyProvider
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+import org.xml.sax.SAXException
+
+class NiFiRegistryAuthorizersXmlEncryptor extends XmlEncryptor {
+
+    private static final Logger logger = LoggerFactory.getLogger(NiFiRegistryAuthorizersXmlEncryptor.class)
+
+    static final String LDAP_USER_GROUP_PROVIDER_CLASS = "org.apache.nifi.registry.security.ldap.tenants.LdapUserGroupProvider"
+    private static final String LDAP_USER_GROUP_PROVIDER_REGEX =
+            /(?s)<userGroupProvider>(?:(?!<userGroupProvider>).)*?<class>\s*org\.apache\.nifi\.registry\.security\.ldap\.tenants\.LdapUserGroupProvider.*?<\/userGroupProvider>/
+    /* Explanation of LDAP_USER_GROUP_PROVIDER_REGEX:
+     *   (?s)                             -> single-line mode (i.e., `.` in regex matches newlines)
+     *   <userGroupProvider>              -> find occurrence of `<userGroupProvider>` literally (case-sensitive)
+     *   (?: ... )                        -> group but do not capture submatch
+     *   (?! ... )                        -> negative lookahead
+     *   (?:(?!<userGroupProvider>).)*?   -> find everything until a new `<userGroupProvider>` starts. This is for not selecting multiple userGroupProviders in one match
+     *   <class>                          -> find occurrence of `<class>` literally (case-sensitive)
+     *   \s*                              -> find any whitespace
+     *   org\.apache\.nifi\.registry\.security\.ldap\.tenants\.LdapUserGroupProvider
+     *                                    -> find occurrence of `org.apache.nifi.registry.security.ldap.tenants.LdapUserGroupProvider` literally (case-sensitive)
+     *   .*?</userGroupProvider>          -> find everything as needed up until and including occurrence of '</userGroupProvider>'
+     */
+
+    NiFiRegistryAuthorizersXmlEncryptor(SensitivePropertyProvider encryptionProvider, SensitivePropertyProvider decryptionProvider) {
+        super(encryptionProvider, decryptionProvider)
+    }
+
+    /**
+     * Overrides the super class implementation to marking xml nodes that should be encrypted.
+     * This is done using logic specific to the authorizers.xml file type targeted by this subclass,
+     * leveraging knowledge of the XML file structure and which elements are sensitive.
+     * Sensitive nodes are marked by adding the encryption="none" attribute.
+     * When all the sensitive values are found and marked, the base class implementation
+     * is invoked to encrypt them.
+     *
+     * @param plainXmlContent the plaintext content of an authorizers.xml file
+     * @return the comment with sensitive values encrypted and marked with the cipher.
+     */
+    @Override
+    String encrypt(String plainXmlContent) {
+        // First, mark the XML nodes to encrypt that are specific to authorizers.xml by adding an attribute encryption="none"
+        String markedXmlContent = markXmlNodesForEncryption(plainXmlContent, "userGroupProvider", {
+            it.find {
+                it.'class' as String == LDAP_USER_GROUP_PROVIDER_CLASS
+            }.property.findAll {
+                // Only operate on populated password properties
+                it.@name =~ "Password" && it.text()
+            }
+        })
+
+        // Now, return the results of the base implementation, which encrypts any node with an encryption="none" attribute
+        return super.encrypt(markedXmlContent)
+    }
+
+    List<String> serializeXmlContentAndPreserveFormat(String updatedXmlContent, String originalXmlContent) {
+        if (updatedXmlContent == originalXmlContent) {
+            // If nothing was encrypted, e.g., the sensitive properties are commented out or empty,
+            // then the best thing to do to preserve formatting perspective is to do nothing.
+            return originalXmlContent.split("\n")
+        }
+
+        // Find & replace the userGroupProvider element of the updated content in the original contents
+        try {
+            def parsedXml = new XmlSlurper().parseText(updatedXmlContent)
+            def provider = parsedXml.userGroupProvider.find { it.'class' as String == LDAP_USER_GROUP_PROVIDER_CLASS }
+            if (provider) {
+                def serializedProvider = new XmlUtil().serialize(provider)
+                // Remove XML declaration from top
+                serializedProvider = serializedProvider.replaceFirst(XML_DECLARATION_REGEX, "")
+                originalXmlContent = originalXmlContent.replaceFirst(LDAP_USER_GROUP_PROVIDER_REGEX, serializedProvider)
+                return originalXmlContent.split("\n")
+            } else {
+                throw new SAXException("No ldap-user-group-provider element found")
+            }
+        } catch (SAXException e) {
+            logger.warn("No userGroupProvider with class ${LDAP_USER_GROUP_PROVIDER_CLASS} found in XML content. " +
+                    "The file could be empty or the element may be missing or commented out")
+            return originalXmlContent.split("\n")
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy
new file mode 100644
index 0000000..fa6ce65
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryIdentityProvidersXmlEncryptor.groovy
@@ -0,0 +1,102 @@
+/*
+ * 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 groovy.xml.XmlUtil
+import org.apache.nifi.properties.SensitivePropertyProvider
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+import org.xml.sax.SAXException
+
+class NiFiRegistryIdentityProvidersXmlEncryptor extends XmlEncryptor {
+
+    private static final Logger logger = LoggerFactory.getLogger(NiFiRegistryIdentityProvidersXmlEncryptor.class)
+
+    static final String LDAP_PROVIDER_CLASS = "org.apache.nifi.registry.security.ldap.LdapIdentityProvider"
+    private static final String LDAP_PROVIDER_REGEX = /(?s)<provider>(?:(?!<provider>).)*?<class>\s*org\.apache\.nifi\.registry\.security\.ldap\.LdapIdentityProvider.*?<\/provider>/
+    /* Explanation of LDAP_PROVIDER_REGEX:
+     *   (?s)                             -> single-line mode (i.e., `.` in regex matches newlines)
+     *   <provider>                       -> find occurrence of `<provider>` literally (case-sensitive)
+     *   (?: ... )                        -> group but do not capture submatch
+     *   (?! ... )                        -> negative lookahead
+     *   (?:(?!<provider>).)*?            -> find everything until a new `<provider>` starts. This is for not selecting multiple providers in one match
+     *   <class>                          -> find occurrence of `<class>` literally (case-sensitive)
+     *   \s*                              -> find any whitespace
+     *   org\.apache\.nifi\.registry\.security\.ldap\.LdapIdentityProvider
+     *                                    -> find occurrence of `org.apache.nifi.registry.security.ldap.LdapIdentityProvider` literally (case-sensitive)
+     *   .*?</provider>                   -> find everything as needed up until and including occurrence of `</provider>`
+     */
+
+    NiFiRegistryIdentityProvidersXmlEncryptor(SensitivePropertyProvider encryptionProvider, SensitivePropertyProvider decryptionProvider) {
+        super(encryptionProvider, decryptionProvider)
+    }
+
+    /**
+     * Overrides the super class implementation to marking xml nodes that should be encrypted.
+     * This is done using logic specific to the identity-providers.xml file type targeted by this
+     * subclass, leveraging knowledge of the XML file structure and which elements are sensitive.
+     * Sensitive nodes are marked by adding the encryption="none" attribute.
+     * When all the sensitive values are found and marked, the base class implementation
+     * is invoked to encrypt them.
+     *
+     * @param plainXmlContent the plaintext content of an identity-providers.xml file
+     * @return the comment with sensitive values encrypted and marked with the cipher.
+     */
+    @Override
+    String encrypt(String plainXmlContent) {
+        // First, mark the XML nodes to encrypt that are specific to authorizers.xml by adding an attribute encryption="none"
+        String markedXmlContent = markXmlNodesForEncryption(plainXmlContent, "provider", {
+            it.find {
+                it.'class' as String == LDAP_PROVIDER_CLASS
+            }.property.findAll {
+                // Only operate on populated password properties
+                it.@name =~ "Password" && it.text()
+            }
+        })
+
+        // Now, return the results of the base implementation, which encrypts any node with an encryption="none" attribute
+        return super.encrypt(markedXmlContent)
+    }
+
+    List<String> serializeXmlContentAndPreserveFormat(String updatedXmlContent, String originalXmlContent) {
+        if (updatedXmlContent == originalXmlContent) {
+            // If nothing was encrypted, e.g., the sensitive properties are commented out or empty,
+            // then the best thing to do to preserve formatting perspective is to do nothing.
+            return originalXmlContent.split("\n")
+        }
+
+        // Find & replace the provider element of the updated content in the original contents
+        try {
+            def parsedXml = new XmlSlurper().parseText(updatedXmlContent)
+            def provider = parsedXml.provider.find { it.'class' as String == LDAP_PROVIDER_CLASS }
+            if (provider) {
+                def serializedProvider = new XmlUtil().serialize(provider)
+                // Remove XML declaration from top
+                serializedProvider = serializedProvider.replaceFirst(XML_DECLARATION_REGEX, "")
+                originalXmlContent = originalXmlContent.replaceFirst(LDAP_PROVIDER_REGEX, serializedProvider)
+                return originalXmlContent.split("\n")
+            } else {
+                throw new SAXException("No ldap-provider found")
+            }
+        } catch (SAXException e) {
+            logger.warn("No provider with class ${LDAP_PROVIDER_CLASS} found in XML content. " +
+                    "The file could be empty or the element may be missing or commented out")
+            return originalXmlContent.split("\n")
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy
new file mode 100644
index 0000000..5ea8d7b
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/NiFiRegistryPropertiesEncryptor.groovy
@@ -0,0 +1,65 @@
+/*
+ * 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.apache.nifi.properties.SensitivePropertyProvider
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import java.util.regex.Pattern
+
+class NiFiRegistryPropertiesEncryptor extends PropertiesEncryptor {
+
+    private static final Logger logger = LoggerFactory.getLogger(NiFiRegistryPropertiesEncryptor.class)
+
+    // TODO, if and when we add a dependency on NiFi Registry, we can import these dependencies array rather than redefining them
+
+    // Defined in nifi-registry-properties: org.apache.nifi.registry.properties.NiFiRegistryProperties
+    private static final String SECURITY_KEYSTORE_PASSWD = "nifi.registry.security.keystorePasswd"
+    private static final String SECURITY_KEY_PASSWD = "nifi.registry.security.keyPasswd"
+    private static final String SECURITY_TRUSTSTORE_PASSWD = "nifi.registry.security.truststorePasswd"
+
+    // Defined in nifi-registry-properties: org.apache.nifi.registry.properties.ProtectedNiFiRegistryProperties
+    private static final String ADDITIONAL_SENSITIVE_PROPERTIES_KEY = "nifi.registry.sensitive.props.additional.keys"
+    private static final String[] DEFAULT_SENSITIVE_PROPERTIES = [
+            SECURITY_KEYSTORE_PASSWD,
+            SECURITY_KEY_PASSWD,
+            SECURITY_TRUSTSTORE_PASSWD
+    ]
+
+    NiFiRegistryPropertiesEncryptor(SensitivePropertyProvider encryptionProvider, SensitivePropertyProvider decryptionProvider) {
+        super(encryptionProvider, decryptionProvider)
+    }
+
+    @Override
+    Properties encrypt(Properties properties) {
+        Set<String> propertiesToEncrypt = new HashSet<>()
+        propertiesToEncrypt.addAll(DEFAULT_SENSITIVE_PROPERTIES)
+        propertiesToEncrypt.addAll(getAdditionalSensitivePropertyKeys(properties))
+
+        return encrypt(properties, propertiesToEncrypt)
+    }
+
+    private static String[] getAdditionalSensitivePropertyKeys(Properties properties) {
+        String rawAdditionalSensitivePropertyKeys = properties.getProperty(ADDITIONAL_SENSITIVE_PROPERTIES_KEY)
+        if (!rawAdditionalSensitivePropertyKeys) {
+            return []
+        }
+        return rawAdditionalSensitivePropertyKeys.split(Pattern.quote(","))
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy
new file mode 100644
index 0000000..189e9a5
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/PropertiesEncryptor.groovy
@@ -0,0 +1,269 @@
+/*
+ * 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 groovy.io.GroovyPrintWriter
+import org.apache.commons.configuration2.PropertiesConfiguration
+import org.apache.commons.configuration2.PropertiesConfigurationLayout
+import org.apache.commons.configuration2.builder.fluent.Configurations
+import org.apache.nifi.properties.SensitivePropertyProvider
+import org.apache.nifi.util.StringUtils
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import java.util.regex.Pattern
+
+class PropertiesEncryptor {
+
+    private static final Logger logger = LoggerFactory.getLogger(PropertiesEncryptor.class)
+
+    private static final String SUPPORTED_PROPERTY_FILE_REGEX = /^\s*nifi\.[-.\w\s]+\s*=/
+    protected static final String PROPERTY_PART_DELIMINATOR = "."
+    protected static final String PROTECTION_ID_PROPERTY_SUFFIX = "protected"
+
+    protected SensitivePropertyProvider encryptionProvider
+    protected SensitivePropertyProvider decryptionProvider
+
+    PropertiesEncryptor(SensitivePropertyProvider encryptionProvider, SensitivePropertyProvider decryptionProvider) {
+        this.encryptionProvider = encryptionProvider
+        this.decryptionProvider = decryptionProvider
+    }
+
+    static boolean supportsFile(String filePath) {
+        try {
+            File file = new File(filePath)
+            if (!ToolUtilities.canRead(file)) {
+                return false
+            }
+            Pattern p = Pattern.compile(SUPPORTED_PROPERTY_FILE_REGEX);
+            return file.readLines().any { it =~ SUPPORTED_PROPERTY_FILE_REGEX }
+        } catch (Throwable ignored) {
+            return false
+        }
+    }
+
+    static Properties loadFile(String filePath) throws IOException {
+
+        Properties rawProperties
+        File inputPropertiesFile = new File(filePath)
+
+        if (ToolUtilities.canRead(inputPropertiesFile)) {
+            rawProperties = new Properties()
+            inputPropertiesFile.withReader { reader ->
+                rawProperties.load(reader)
+            }
+        } else {
+            throw new IOException("The file at ${filePath} must exist and be readable by the user running this tool")
+        }
+
+        return rawProperties
+
+    }
+
+    Properties decrypt(final Properties properties) {
+
+        Set<String> propertiesToSkip = getProtectionIdPropertyKeys(properties)
+        Map<String, String> propertiesToDecrypt = getProtectedPropertyKeys(properties)
+
+        if (propertiesToDecrypt.isEmpty()) {
+            return properties
+        }
+
+        if (decryptionProvider == null) {
+            throw new IllegalStateException("Decryption capability not supported without provider. " +
+                    "Usually this means a decryption password / key was not provided to the tool.")
+        }
+
+        String supportedDecryptionScheme = decryptionProvider.getIdentifierKey()
+        if (supportedDecryptionScheme) {
+            propertiesToDecrypt.entrySet().each { entry ->
+                if (!supportedDecryptionScheme.equals(entry.getValue())) {
+                    throw new IllegalStateException("Decryption capability not supported by this tool. " +
+                            "This tool supports ${supportedDecryptionScheme}, but this properties file contains " +
+                            "${entry.getKey()} protected by ${entry.getValue()}")
+                }
+            }
+        }
+
+        Properties unprotectedProperties = new Properties()
+
+        for (String propertyName : properties.stringPropertyNames()) {
+            String propertyValue = properties.getProperty(propertyName)
+            if (propertiesToSkip.contains(propertyName)) {
+                continue
+            }
+            if (propertiesToDecrypt.keySet().contains(propertyName)) {
+                String decryptedPropertyValue = decryptionProvider.unprotect(propertyValue)
+                unprotectedProperties.setProperty(propertyName, decryptedPropertyValue)
+            } else {
+                unprotectedProperties.setProperty(propertyName, propertyValue)
+            }
+        }
+
+        return unprotectedProperties
+    }
+
+    Properties encrypt(Properties properties) {
+        return encrypt(properties, properties.stringPropertyNames())
+    }
+
+    Properties encrypt(final Properties properties, final Set<String> propertiesToEncrypt) {
+
+        if (encryptionProvider == null) {
+            throw new IllegalStateException("Input properties is encrypted, but decryption capability is not enabled. " +
+                    "Usually this means a decryption password / key was not provided to the tool.")
+        }
+
+        logger.debug("Encrypting ${propertiesToEncrypt.size()} properties")
+
+        Properties protectedProperties = new Properties();
+        for (String propertyName : properties.stringPropertyNames()) {
+            String propertyValue = properties.getProperty(propertyName)
+            // empty properties are not encrypted
+            if (!StringUtils.isEmpty(propertyValue) && propertiesToEncrypt.contains(propertyName)) {
+                String encryptedPropertyValue = encryptionProvider.protect(propertyValue)
+                protectedProperties.setProperty(propertyName, encryptedPropertyValue)
+                protectedProperties.setProperty(protectionPropertyForProperty(propertyName), encryptionProvider.getIdentifierKey())
+            } else {
+                protectedProperties.setProperty(propertyName, propertyValue)
+            }
+        }
+
+        return protectedProperties
+    }
+
+    void write(Properties updatedProperties, String outputFilePath, String inputFilePath) {
+        if (!outputFilePath) {
+            throw new IllegalArgumentException("Cannot write encrypted properties to empty file path")
+        }
+        File outputPropertiesFile = new File(outputFilePath)
+
+        if (ToolUtilities.isSafeToWrite(outputPropertiesFile)) {
+            String serializedProperties = serializePropertiesAndPreserveFormatIfPossible(updatedProperties, inputFilePath)
+            outputPropertiesFile.text = serializedProperties
+        } else {
+            throw new IOException("The file at ${outputFilePath} must be writable by the user running this tool")
+        }
+    }
+
+    private String serializePropertiesAndPreserveFormatIfPossible(Properties updatedProperties, String inputFilePath) {
+        List<String> linesToPersist
+        File inputPropertiesFile = new File(inputFilePath)
+        if (ToolUtilities.canRead(inputPropertiesFile)) {
+            // Instead of just writing the Properties instance to a properties file,
+            // this method attempts to maintain the structure of the original file and preserves comments
+            linesToPersist = serializePropertiesAndPreserveFormat(updatedProperties, inputPropertiesFile)
+        } else {
+            linesToPersist = serializeProperties(updatedProperties)
+        }
+        return linesToPersist.join("\n")
+    }
+
+    private List<String> serializePropertiesAndPreserveFormat(Properties properties, File originalPropertiesFile) {
+        Configurations configurations = new Configurations()
+        try {
+            PropertiesConfiguration originalPropertiesConfiguration = configurations.properties(originalPropertiesFile)
+            def keysToAdd = properties.keySet().findAll { !originalPropertiesConfiguration.containsKey(it.toString()) }
+            def keysToUpdate = properties.keySet().findAll {
+                !keysToAdd.contains(it) &&
+                        properties.getProperty(it.toString()) != originalPropertiesConfiguration.getProperty(it.toString())
+            }
+            def keysToRemove = originalPropertiesConfiguration.getKeys().findAll {!properties.containsKey(it) }
+
+            keysToUpdate.forEach {
+                originalPropertiesConfiguration.setProperty(it.toString(), properties.getProperty(it.toString()))
+            }
+            keysToRemove.forEach {
+                originalPropertiesConfiguration.clearProperty(it.toString())
+            }
+            boolean isFirst = true
+            keysToAdd.sort().forEach {
+                originalPropertiesConfiguration.setProperty(it.toString(), properties.getProperty(it.toString()))
+                if (isFirst) {
+                    originalPropertiesConfiguration.getLayout().setBlancLinesBefore(it.toString(), 1)
+                    originalPropertiesConfiguration.getLayout().setComment(it.toString(), "protection properties")
+                    isFirst = false
+                }
+            }
+
+            OutputStream out = new ByteArrayOutputStream()
+            Writer writer = new GroovyPrintWriter(out)
+
+            PropertiesConfigurationLayout layout = originalPropertiesConfiguration.getLayout()
+            layout.setGlobalSeparator("=")
+            layout.save(originalPropertiesConfiguration, writer)
+
+            writer.flush()
+            List<String> lines = out.toString().split("\n")
+
+            return lines
+        } catch(Exception e) {
+            throw new RuntimeException("Error serializing properties.", e)
+        }
+    }
+
+    private List<String> serializeProperties(final Properties properties) {
+        OutputStream out = new ByteArrayOutputStream()
+        Writer writer = new GroovyPrintWriter(out)
+
+        properties.store(writer, null)
+        writer.flush()
+        List<String> lines = out.toString().split("\n")
+
+        return lines
+    }
+
+    /**
+     * Returns a Map of the keys identifying properties that are currently protected
+     * and the protection identifier for each. The protection
+     *
+     * @return the Map of protected property keys and the protection identifier for each
+     */
+    private static Map<String, String> getProtectedPropertyKeys(Properties properties) {
+        Map<String, String> protectedProperties = new HashMap<>();
+        properties.stringPropertyNames().forEach({ key ->
+            String protectionKey = protectionPropertyForProperty(key)
+            String protectionIdentifier = properties.getProperty(protectionKey)
+            if (protectionIdentifier) {
+                protectedProperties.put(key, protectionIdentifier)
+            }
+        })
+        return protectedProperties
+    }
+
+    private static Set<String> getProtectionIdPropertyKeys(Properties properties) {
+        Set<String> protectedProperties = properties.stringPropertyNames().findAll { key ->
+            key.endsWith(PROPERTY_PART_DELIMINATOR + PROTECTION_ID_PROPERTY_SUFFIX)
+        }
+        return protectedProperties;
+    }
+
+    private static String protectionPropertyForProperty(String propertyName) {
+        return propertyName + PROPERTY_PART_DELIMINATOR + PROTECTION_ID_PROPERTY_SUFFIX
+    }
+
+    private static String propertyForProtectionProperty(String protectionPropertyName) {
+        String[] propertyNameParts = protectionPropertyName.split(Pattern.quote(PROPERTY_PART_DELIMINATOR))
+        if (propertyNameParts.length >= 2 && PROTECTION_ID_PROPERTY_SUFFIX.equals(propertyNameParts[-1])) {
+            return propertyNameParts[(0..-2)].join(PROPERTY_PART_DELIMINATOR)
+        }
+        return null
+    }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/ToolUtilities.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/ToolUtilities.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/ToolUtilities.groovy
new file mode 100644
index 0000000..4e8c1b7
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/ToolUtilities.groovy
@@ -0,0 +1,164 @@
+/*
+ * 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.apache.commons.cli.CommandLine
+import org.apache.commons.codec.binary.Hex
+import org.apache.nifi.util.console.TextDevice
+import org.apache.nifi.util.console.TextDevices
+import org.bouncycastle.crypto.generators.SCrypt
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.crypto.Cipher
+import java.nio.charset.StandardCharsets
+import java.security.KeyException
+
+class ToolUtilities {
+
+    private static final Logger logger = LoggerFactory.getLogger(ToolUtilities.class)
+
+    private static final int DEFAULT_MIN_PASSWORD_LENGTH = 12
+
+    // Strong parameters as of 12 Aug 2016
+    private static final int SCRYPT_N = 2**16
+    private static final int SCRYPT_R = 8
+    private static final int SCRYPT_P = 1
+
+    static boolean isExactlyOneOptionSet(CommandLine commandLine, String... opt) {
+        Collection<Boolean> setOptions = opt.findAll{commandLine.hasOption(it)}
+        return setOptions.size() == 1
+    }
+
+    static boolean isExactlyOneTrue(Boolean... b) {
+        Collection<Boolean> trues = b.findAll{it}
+        return trues.size() == 1
+    }
+
+    /**
+     * Helper method which returns true if the provided file exists and is readable
+     *
+     * @param fileToRead the proposed file to read
+     * @return true if the caller should be able to successfully read from this file
+     */
+    static boolean canRead(File fileToRead) {
+        fileToRead && (fileToRead.exists() && fileToRead.canRead())
+    }
+
+    /**
+     * Helper method which returns true if it is "safe" to write to the provided file.
+     *
+     * Conditions:
+     *  file does not exist and the parent directory is writable
+     *  -OR-
+     *  file exists and is writable
+     *
+     * @param fileToWrite the proposed file to be written to
+     * @return true if the caller can "safely" write to this file location
+     */
+    static boolean isSafeToWrite(File fileToWrite) {
+        fileToWrite && ((!fileToWrite.exists() && fileToWrite.absoluteFile.parentFile.canWrite()) || (fileToWrite.exists() && fileToWrite.canWrite()))
+    }
+
+
+    /**
+     * The method returns the provided, derived, or securely-entered key in hex format.
+     *
+     * @param device
+     * @param keyHex
+     * @param password
+     * @param usingPassword
+     * @return
+     */
+    public static String determineKey(TextDevice device = TextDevices.defaultTextDevice(), String keyHex, String password, boolean usingPassword) {
+        if (usingPassword) {
+            if (!password) {
+                logger.debug("Reading password from secure console")
+                password = readPasswordFromConsole(device)
+            }
+            keyHex = deriveKeyFromPassword(password)
+            password = null
+            return keyHex
+        } else {
+            if (!keyHex) {
+                logger.debug("Reading hex key from secure console")
+                keyHex = readKeyFromConsole(device)
+            }
+            return keyHex
+        }
+    }
+
+    private static String readKeyFromConsole(TextDevice textDevice) {
+        textDevice.printf("Enter the master key in hexadecimal format (spaces acceptable): ")
+        new String(textDevice.readPassword())
+    }
+
+    private static String readPasswordFromConsole(TextDevice textDevice) {
+        textDevice.printf("Enter the password: ")
+        new String(textDevice.readPassword())
+    }
+
+//    /**
+//     * 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.
+//     *
+//     * @param rawKey the unprocessed key input
+//     * @return the formatted hex string in uppercase
+//     * @throws java.security.KeyException if the key is not a valid length after parsing
+//     */
+//    public static String parseKey(String rawKey) throws KeyException {
+//        String hexKey = rawKey.replaceAll("[^0-9a-fA-F]", "")
+//        def validKeyLengths = getValidKeyLengths()
+//        if (!validKeyLengths.contains(hexKey.size() * 4)) {
+//            throw new KeyException("The key (${hexKey.size()} hex chars) must be of length ${validKeyLengths} bits (${validKeyLengths.collect { it / 4 }} hex characters)")
+//        }
+//        hexKey.toUpperCase()
+//    }
+
+    /**
+     * Returns the list of acceptable key lengths in bits based on the current JCE policies.
+     *
+     * @return 128 , [192, 256]
+     */
+    public static List<Integer> getValidKeyLengths() {
+        Cipher.getMaxAllowedKeyLength("AES") > 128 ? [128, 192, 256] : [128]
+    }
+
+    private static String deriveKeyFromPassword(String password, int minPasswordLength = DEFAULT_MIN_PASSWORD_LENGTH) {
+        password = password?.trim()
+        if (!password || password.length() < minPasswordLength) {
+            throw new KeyException("Cannot derive key from empty/short password -- password must be at least ${minPasswordLength} characters")
+        }
+
+        // Generate a 128 bit salt
+        byte[] salt = generateScryptSalt()
+        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() {
+//        byte[] salt = new byte[16]
+//        new SecureRandom().nextBytes(salt)
+//        salt
+        /* It is not ideal to use a static salt, but the KDF operation must be deterministic
+        for a given password, and storing and retrieving the salt in bootstrap.conf causes
+        compatibility concerns
+        */
+        "NIFI_SCRYPT_SALT".getBytes(StandardCharsets.UTF_8)
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy
new file mode 100644
index 0000000..8324682
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/groovy/org/apache/nifi/toolkit/encryptconfig/util/XmlEncryptor.groovy
@@ -0,0 +1,200 @@
+/*
+ * 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 groovy.util.slurpersupport.GPathResult
+import groovy.xml.XmlUtil
+import org.apache.nifi.properties.SensitivePropertyProvider
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+abstract class XmlEncryptor {
+
+    protected static final String XML_DECLARATION_REGEX = /<\?xml version="1.0" encoding="UTF-8"\?>/
+    protected static final ENCRYPTION_NONE = "none"
+    protected static final ENCRYPTION_EMPTY = ""
+
+    private static final Logger logger = LoggerFactory.getLogger(XmlEncryptor.class)
+
+    protected SensitivePropertyProvider decryptionProvider
+    protected SensitivePropertyProvider encryptionProvider
+
+    XmlEncryptor(SensitivePropertyProvider encryptionProvider, SensitivePropertyProvider decryptionProvider) {
+        this.decryptionProvider = decryptionProvider
+        this.encryptionProvider = encryptionProvider
+    }
+
+    static boolean supportsFile(String filePath) {
+        def doc
+        try {
+            String rawFileContents = loadXmlFile(filePath)
+            doc = new XmlSlurper().parseText(rawFileContents)
+        } catch (Throwable ignored) {
+            return false
+        }
+        return doc != null
+    }
+
+    static String loadXmlFile(String xmlFilePath) throws IOException {
+        File xmlFile = new File(xmlFilePath)
+        if (ToolUtilities.canRead(xmlFile)) {
+            try {
+                String xmlContent = xmlFile.text
+                return xmlContent
+            } catch (RuntimeException e) {
+                throw new IOException("Cannot load XML from ${xmlFilePath}", e)
+            }
+        } else {
+            throw new IOException("File at ${xmlFilePath} must exist and be readable by user running this tool.")
+        }
+    }
+
+    String decrypt(String encryptedXmlContent) {
+        try {
+
+            def doc = new XmlSlurper().parseText(encryptedXmlContent)
+            GPathResult[] encryptedNodes = doc.depthFirst().findAll { GPathResult node ->
+                node.@encryption != ENCRYPTION_NONE && node.@encryption != ENCRYPTION_EMPTY
+            }
+
+            if (encryptedNodes.size() == 0) {
+                return encryptedXmlContent
+            }
+
+            if (decryptionProvider == null) {
+                throw new IllegalStateException("Input XML is encrypted, but decryption capability is not enabled. " +
+                        "Usually this means a decryption password / key was not provided to the tool.")
+            }
+            String supportedDecryptionScheme = decryptionProvider.getIdentifierKey()
+
+            logger.debug("Found ${encryptedNodes.size()} encrypted XML elements. Will attempt to decrypt using the provided decryption key.")
+
+            encryptedNodes.each { node ->
+                logger.debug("Attempting to decrypt ${node.text()}")
+                if (node.@encryption != supportedDecryptionScheme) {
+                    throw new IllegalStateException("Decryption capability not supported by this tool. " +
+                            "This tool supports ${supportedDecryptionScheme}, but this xml file contains " +
+                            "${node.toString()} protected by ${node.@encryption}")
+                }
+                String decryptedValue = decryptionProvider.unprotect(node.text().trim())
+                node.@encryption = ENCRYPTION_NONE
+                node.replaceBody(decryptedValue)
+            }
+
+            // Does not preserve whitespace formatting or comments
+            String updatedXml = XmlUtil.serialize(doc)
+            logger.debug("Updated XML content: ${updatedXml}")
+            return updatedXml
+
+        } catch (Exception e) {
+            throw new RuntimeException("Cannot decrypt XML content", e)
+        }
+    }
+
+    String encrypt(String plainXmlContent) {
+        try {
+            def doc = new XmlSlurper().parseText(plainXmlContent)
+
+            GPathResult[] nodesToEncrypt = doc.depthFirst().findAll { GPathResult node ->
+                node.text() && node.@encryption == ENCRYPTION_NONE
+            }
+
+            logger.debug("Encrypting ${nodesToEncrypt.size()} element(s) of XML decoument")
+
+            if (nodesToEncrypt.size() == 0) {
+                return plainXmlContent
+            }
+
+            nodesToEncrypt.each { node ->
+                String encryptedValue = this.encryptionProvider.protect(node.text().trim())
+                node.@encryption = this.encryptionProvider.getIdentifierKey()
+                node.replaceBody(encryptedValue)
+            }
+
+            // Does not preserve whitespace formatting or comments
+            String updatedXml = XmlUtil.serialize(doc)
+            logger.debug("Updated XML content: ${updatedXml}")
+            return updatedXml
+        } catch (Exception e) {
+            throw new RuntimeException("Cannot encrypt XML content", e)
+        }
+    }
+
+    void writeXmlFile(String updatedXmlContent, String outputXmlPath, String inputXmlPath) throws IOException {
+        File outputXmlFile = new File(outputXmlPath)
+        if (ToolUtilities.isSafeToWrite(outputXmlFile)) {
+            String finalXmlContent = serializeXmlContentAndPreserveFormatIfPossible(updatedXmlContent, inputXmlPath)
+            outputXmlFile.text = finalXmlContent
+        } else {
+            throw new IOException("The XML file at ${outputXmlPath} must be writable by the user running this tool")
+        }
+    }
+
+    String serializeXmlContentAndPreserveFormatIfPossible(String updatedXmlContent, String inputXmlPath) {
+        String finalXmlContent
+        File inputXmlFile = new File(inputXmlPath)
+        if (ToolUtilities.canRead(inputXmlFile)) {
+            String originalXmlContent = new File(inputXmlPath).text
+            // Instead of just writing the XML content to a file, this method attempts to maintain
+            // the structure of the original file.
+            finalXmlContent = serializeXmlContentAndPreserveFormat(updatedXmlContent, originalXmlContent).join("\n")
+        } else {
+            finalXmlContent = updatedXmlContent
+        }
+        return finalXmlContent
+    }
+
+    /**
+     * Given an original XML file and updated XML content, create the lines for an updated, minimally altered, serialization.
+     * Concrete classes extending this class must implement this method using specific knowledge of the XML document.
+     *
+     * @param finalXmlContent the xml content to serialize
+     * @param inputXmlFile the original input xml file to use as a template for formatting the serialization
+     * @return the lines of serialized finalXmlContent that are close in raw format to originalInputXmlFile
+     */
+    abstract List<String> serializeXmlContentAndPreserveFormat(String updatedXmlContent, String originalXmlContent)
+    // TODO, replace the above abstract method with an implementation that works generically for any updated (encryption=."..") nodes
+    // perhaps this could be done leveraging org.apache.commons.configuration2 which is capable of preserving comments, eg:
+
+
+    static String markXmlNodesForEncryption(String plainXmlContent, String gPath, gPathCallback) {
+        String markedXmlContent
+        try {
+            def doc = new XmlSlurper().parseText(plainXmlContent)
+            // Find the provider element by class even if it has been renamed
+            def sensitiveProperties = gPathCallback(doc."${gPath}")
+
+            logger.debug("Marking ${sensitiveProperties.size()} sensitive element(s) of XML to be encrypted")
+
+            if (sensitiveProperties.size() == 0) {
+                logger.debug("No populated sensitive properties found in XML content")
+                return plainXmlContent
+            }
+
+            sensitiveProperties.each {
+                it.@encryption = ENCRYPTION_NONE
+            }
+
+            // Does not preserve whitespace formatting or comments
+            // TODO: Switch to XmlParser & XmlNodePrinter to maintain "empty" element structure
+            markedXmlContent = XmlUtil.serialize(doc)
+        } catch (Exception e) {
+            logger.debug("Encountered exception", e)
+            throw new RuntimeException(e)
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/a8817e02/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/resources/log4j.properties b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/resources/log4j.properties
index fc2aaf1..9a13479 100644
--- a/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/resources/log4j.properties
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/main/resources/log4j.properties
@@ -18,5 +18,6 @@
 log4j.rootLogger=INFO,console
 
 log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.Target=System.err
 log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
\ No newline at end of file
+log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %p %c{1}: %m%n

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/EncryptConfigMainTest.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/EncryptConfigMainTest.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/EncryptConfigMainTest.groovy
new file mode 100644
index 0000000..b2b8040
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/EncryptConfigMainTest.groovy
@@ -0,0 +1,285 @@
+/*
+ * 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 org.apache.nifi.properties.AESSensitivePropertyProvider
+import org.apache.nifi.properties.ConfigEncryptionTool
+import org.apache.nifi.properties.NiFiPropertiesLoader
+import org.apache.nifi.toolkit.encryptconfig.util.BootstrapUtil
+import org.apache.nifi.util.NiFiProperties
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.BeforeClass
+import org.junit.Rule
+import org.junit.Test
+import org.junit.contrib.java.lang.system.Assertion
+import org.junit.contrib.java.lang.system.ExpectedSystemExit
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import java.nio.file.Files
+import java.security.Security
+
+import static org.apache.nifi.toolkit.encryptconfig.TestUtil.*
+
+@RunWith(JUnit4.class)
+class EncryptConfigMainTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(EncryptConfigMainTest.class)
+
+    @Rule
+    public final ExpectedSystemExit exit = ExpectedSystemExit.none()
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+
+        setupTmpDir()
+    }
+
+    @Test
+    void testDetermineModeFromArgsWithLegacyMode() {
+        // Arrange
+        def argsList = "-b conf/bootstrap.conf -n conf/nifi.properties".split(" ").toList()
+
+        // Act
+        def toolMode = EncryptConfigMain.determineModeFromArgs(argsList)
+
+        // Assert
+        toolMode != null
+        toolMode instanceof LegacyMode
+    }
+
+    @Test
+    void testDetermineModeFromArgsWithNifiRegistryMode() {
+        // Arrange
+        def argsList = "--nifiRegistry".split(" ").toList()
+        // Act
+        def toolMode = EncryptConfigMain.determineModeFromArgs(argsList)
+        // Assert
+        toolMode != null
+        toolMode instanceof NiFiRegistryMode
+        !argsList.contains("--nifiRegistry")
+
+        /* Test when --nifiRegistry is not first flag */
+
+        // Arrange
+        argsList = "-b conf/bootstrap.conf -p --nifiRegistry -r conf/nifi-registry.properties".split(" ").toList()
+        // Act
+        toolMode = EncryptConfigMain.determineModeFromArgs(argsList)
+        // Assert
+        toolMode != null
+        toolMode instanceof NiFiRegistryMode
+        !argsList.contains("--nifiRegistry")
+    }
+
+    @Test
+    void testDetermineModeFromArgsWithNifiRegistryDecryptMode() {
+        // Arrange
+        def argsList = "--nifiRegistry --decrypt".split(" ").toList()
+        // Act
+        def toolMode = EncryptConfigMain.determineModeFromArgs(argsList)
+        // Assert
+        toolMode != null
+        toolMode instanceof NiFiRegistryDecryptMode
+        !argsList.contains("--nifiRegistry")
+        !argsList.contains("--decrypt")
+
+        /* Test when --decrypt comes before --nifiRegistry  */
+
+        // Arrange
+        argsList = "--b conf/bootstrap.conf --decrypt --nifiRegistry".split(" ").toList()
+        // Act
+        toolMode = EncryptConfigMain.determineModeFromArgs(argsList)
+        // Assert
+        toolMode != null
+        toolMode instanceof NiFiRegistryDecryptMode
+        !argsList.contains("--nifiRegistry")
+        !argsList.contains("--decrypt")
+    }
+
+    @Test
+    void testDetermineModeFromArgsReturnsNullOnDecryptWithoutNifiRegistryPresent() {
+        // Arrange
+        def argsList = "--decrypt".split(" ").toList()
+
+        // Act
+        def toolMode = EncryptConfigMain.determineModeFromArgs(argsList)
+
+        // Assert
+        toolMode == null
+    }
+
+    @Test
+    void testShouldPerformFullOperationForNiFiPropertiesAndLoginIdentityProvidersAndAuthorizers() {
+        // Arrange
+        exit.expectSystemExitWithStatus(0)
+
+        File tmpDir = setupTmpDir()
+
+        File emptyKeyFile = new File("src/test/resources/bootstrap_with_empty_master_key.conf")
+        File bootstrapFile = new File("target/tmp/tmp_bootstrap.conf")
+        bootstrapFile.delete()
+
+        Files.copy(emptyKeyFile.toPath(), bootstrapFile.toPath())
+        final List<String> originalBootstrapLines = bootstrapFile.readLines()
+        String originalKeyLine = originalBootstrapLines.find {
+            it.startsWith("${BootstrapUtil.NIFI_BOOTSTRAP_KEY_PROPERTY}=")
+        }
+        logger.info("Original key line from bootstrap.conf: ${originalKeyLine}")
+        assert originalKeyLine == "${BootstrapUtil.NIFI_BOOTSTRAP_KEY_PROPERTY}="
+
+        final String EXPECTED_KEY_LINE = "${BootstrapUtil.NIFI_BOOTSTRAP_KEY_PROPERTY}=${KEY_HEX}"
+
+        // Set up the NFP file
+        File inputPropertiesFile = new File("src/test/resources/nifi_with_sensitive_properties_unprotected.properties")
+        File outputPropertiesFile = new File("target/tmp/tmp_nifi.properties")
+        outputPropertiesFile.delete()
+
+        NiFiProperties inputProperties = new NiFiPropertiesLoader().load(inputPropertiesFile)
+        logger.info("Loaded ${inputProperties.size()} properties from input file")
+
+        // Set up the LIP file
+        File inputLIPFile = new File("src/test/resources/login-identity-providers-populated.xml")
+        File outputLIPFile = new File("target/tmp/tmp-lip.xml")
+        outputLIPFile.delete()
+
+        String originalLipXmlContent = inputLIPFile.text
+        logger.info("Original LIP XML content: ${originalLipXmlContent}")
+
+        // Set up the Authorizers file
+        File inputAuthorizersFile = new File("src/test/resources/authorizers-populated.xml")
+        File outputAuthorizersFile = new File("target/tmp/tmp-authorizers.xml")
+        outputAuthorizersFile.delete()
+
+        String originalAuthorizersXmlContent = inputAuthorizersFile.text
+        logger.info("Original Authorizers XML content: ${originalAuthorizersXmlContent}")
+
+        String[] args = [
+                "-n", inputPropertiesFile.path,
+                "-l", inputLIPFile.path,
+                "-a", inputAuthorizersFile.path,
+                "-b", bootstrapFile.path,
+                "-o", outputPropertiesFile.path,
+                "-i", outputLIPFile.path,
+                "-u", outputAuthorizersFile.path,
+                "-k", KEY_HEX,
+                "-v"]
+
+        AESSensitivePropertyProvider spp = new AESSensitivePropertyProvider(KEY_HEX)
+
+        exit.checkAssertionAfterwards(new Assertion() {
+            void checkAssertion() {
+
+                /*** NiFi Properties Assertions ***/
+
+                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 encrypted)
+                NiFiProperties updatedProperties = new NiFiPropertiesLoader().readProtectedPropertiesFromDisk(outputPropertiesFile)
+                assert updatedProperties.size() >= inputProperties.size()
+
+                // 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())
+                }
+
+                /*** Login Identity Providers Assertions ***/
+
+                final String updatedLipXmlContent = outputLIPFile.text
+                logger.info("Updated LIP XML content: ${updatedLipXmlContent}")
+                // Check that the output values for sensitive properties are not the same as the original (i.e. it was encrypted)
+                def originalLipParsedXml = new XmlSlurper().parseText(originalLipXmlContent)
+                def updatedLipParsedXml = new XmlSlurper().parseText(updatedLipXmlContent)
+                assert originalLipParsedXml != updatedLipParsedXml
+                assert originalLipParsedXml.'**'.findAll { it.@encryption } != updatedLipParsedXml.'**'.findAll {
+                    it.@encryption
+                }
+                def lipEncryptedValues = updatedLipParsedXml.provider.find {
+                    it.identifier == 'ldap-provider'
+                }.property.findAll {
+                    it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
+                }
+                lipEncryptedValues.each {
+                    assert spp.unprotect(it.text()) == PASSWORD
+                }
+                // Check that the comments are still there
+                def lipTrimmedLines = inputLIPFile.readLines().collect { it.trim() }.findAll { it }
+                def lipTrimmedSerializedLines = updatedLipXmlContent.split("\n").collect { it.trim() }.findAll { it }
+                assert lipTrimmedLines.size() == lipTrimmedSerializedLines.size()
+
+                /*** Authorizers Assertions ***/
+
+                final String updatedAuthorizersXmlContent = outputAuthorizersFile.text
+                logger.info("Updated Authorizers XML content: ${updatedAuthorizersXmlContent}")
+                // Check that the output values for sensitive properties are not the same as the original (i.e. it was encrypted)
+                def originalAuthorizersParsedXml = new XmlSlurper().parseText(originalAuthorizersXmlContent)
+                def updatedAuthorizersParsedXml = new XmlSlurper().parseText(updatedAuthorizersXmlContent)
+                assert originalAuthorizersParsedXml != updatedAuthorizersParsedXml
+                assert originalAuthorizersParsedXml.'**'.findAll { it.@encryption } != updatedAuthorizersParsedXml.'**'.findAll {
+                    it.@encryption
+                }
+                def authorizersEncryptedValues = updatedAuthorizersParsedXml.userGroupProvider.find {
+                    it.identifier == 'ldap-user-group-provider'
+                }.property.findAll {
+                    it.@name =~ "Password" && it.@encryption =~ "aes/gcm/\\d{3}"
+                }
+                authorizersEncryptedValues.each {
+                    assert spp.unprotect(it.text()) == PASSWORD
+                }
+                // 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 }
+                assert authorizersTrimmedLines.size() == authorizersTrimmedSerializedLines.size()
+
+                /*** Bootstrap assertions ***/
+
+                // Check that the key was persisted to the bootstrap.conf
+                final List<String> updatedBootstrapLines = bootstrapFile.readLines()
+                String updatedKeyLine = updatedBootstrapLines.find {
+                    it.startsWith(BootstrapUtil.NIFI_BOOTSTRAP_KEY_PROPERTY)
+                }
+                logger.info("Updated key line: ${updatedKeyLine}")
+
+                assert updatedKeyLine == EXPECTED_KEY_LINE
+                assert originalBootstrapLines.size() == updatedBootstrapLines.size()
+
+                // Clean up
+                outputPropertiesFile.deleteOnExit()
+                outputLIPFile.deleteOnExit()
+                outputAuthorizersFile.deleteOnExit()
+                bootstrapFile.deleteOnExit()
+                tmpDir.deleteOnExit()
+            }
+        })
+
+        // Act
+        EncryptConfigMain.main(args)
+        logger.info("Invoked #main with ${args.join(" ")}")
+
+        // Assert
+
+        // Assertions defined above
+    }
+
+}

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/NiFiRegistryDecryptModeSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryDecryptModeSpec.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryDecryptModeSpec.groovy
new file mode 100644
index 0000000..2ff1414
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryDecryptModeSpec.groovy
@@ -0,0 +1,117 @@
+/*
+ * 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 org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.Assume
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+import spock.lang.Specification
+
+import java.security.Security
+
+import static org.apache.nifi.toolkit.encryptconfig.TestUtil.*
+
+class NiFiRegistryDecryptModeSpec extends Specification {
+    private static final Logger logger = LoggerFactory.getLogger(NiFiRegistryDecryptModeSpec.class)
+
+    ByteArrayOutputStream toolStdOutContent
+    PrintStream origSystemOut
+
+    // runs before every feature method
+    def setup() {
+        origSystemOut = System.out
+        toolStdOutContent = new ByteArrayOutputStream();
+        System.setOut(new PrintStream(toolStdOutContent));
+    }
+
+    // runs after every feature method
+    def cleanup() {
+        toolStdOutContent.flush()
+        System.setOut(origSystemOut);
+        toolStdOutContent.close()
+    }
+
+    // runs before the first feature method
+    def setupSpec() {
+        Security.addProvider(new BouncyCastleProvider())
+        setupTmpDir()
+    }
+
+    // runs after the last feature method
+    def cleanupSpec() {
+        cleanupTmpDir()
+    }
+
+    def "decrypt protected nifi-registry.properties file using -k"() {
+
+        setup:
+        NiFiRegistryDecryptMode tool = new NiFiRegistryDecryptMode()
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128)
+        File outRegistryProperties1 = generateTmpFile()
+
+        when: "run with args: -k <key> -r <file>"
+        tool.run("-k ${KEY_HEX_128} -r ${inRegistryProperties1}".split(" "))
+        toolStdOutContent.flush()
+        outRegistryProperties1.text = toolStdOutContent.toString()
+        then: "decrypted properties file was printed to std out"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED, outRegistryProperties1.getAbsolutePath(), true)
+        and: "input properties file is still encrypted"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128, inRegistryProperties1, true)
+
+    }
+
+    def "decrypt protected nifi-registry.properties file using -p [256-bit]"() {
+
+        Assume.assumeTrue("Test only runs when unlimited strength crypto is available", isUnlimitedStrengthCryptoAvailable())
+
+        setup:
+        NiFiRegistryDecryptMode tool = new NiFiRegistryDecryptMode()
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_PASSWORD_256)
+        File outRegistryProperties1 = generateTmpFile()
+
+        when: "run with args: -p <password> -r <file>"
+        tool.run("-p ${PASSWORD} -r ${inRegistryProperties1}".split(" "))
+        toolStdOutContent.flush()
+        outRegistryProperties1.text = toolStdOutContent.toString()
+        then: "decrypted properties file was printed to std out"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED, outRegistryProperties1.getAbsolutePath(), true)
+        and: "input properties file is still encrypted"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_PASSWORD_256, inRegistryProperties1, true)
+
+    }
+
+    def "decrypt protected nifi-registry.properties file using -b"() {
+
+        setup:
+        NiFiRegistryDecryptMode tool = new NiFiRegistryDecryptMode()
+        def inRegistryProperties = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128)
+        def inBootstrap = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128)
+        File outRegistryProperties = generateTmpFile()
+
+        when: "run with args: -b <file> -r <file>"
+        tool.run("-b ${inBootstrap} -r ${inRegistryProperties}".split(" "))
+        toolStdOutContent.flush()
+        outRegistryProperties.text = toolStdOutContent.toString()
+        then: "decrypted properties file was printed to std out"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED, outRegistryProperties.getAbsolutePath(), true)
+        and: "input properties file is still encrypted"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_POPULATED_PROTECTED_KEY_128, inRegistryProperties, 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/NiFiRegistryModeSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryModeSpec.groovy b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryModeSpec.groovy
new file mode 100644
index 0000000..137eeea
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-encrypt-config/src/test/groovy/org/apache/nifi/toolkit/encryptconfig/NiFiRegistryModeSpec.groovy
@@ -0,0 +1,331 @@
+/*
+ * 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 org.apache.nifi.toolkit.encryptconfig.util.BootstrapUtil
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+import spock.lang.Specification
+
+import java.security.Security
+
+import static org.apache.nifi.toolkit.encryptconfig.TestUtil.*
+
+class NiFiRegistryModeSpec extends Specification {
+    private static final Logger logger = LoggerFactory.getLogger(NiFiRegistryModeSpec.class)
+
+    // runs before every feature method
+    def setup() {}
+
+    // runs after every feature method
+    def cleanup() {}
+
+    // runs before the first feature method
+    def setupSpec() {
+        Security.addProvider(new BouncyCastleProvider())
+        setupTmpDir()
+    }
+
+    // runs after the last feature method
+    def cleanupSpec() {
+        cleanupTmpDir()
+    }
+
+    def "writing key to bootstrap.conf file"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inBootstrapConf3 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def outBootstrapConf3 = generateTmpFilePath()
+        def inBootstrapConf4 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def outBootstrapConf4 = generateTmpFilePath()
+        def inBootstrapConf5 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128)
+        def outBootstrapConf5 = generateTmpFilePath()
+
+        when: "run with args: -k <key> -b <file>"
+        tool.run("-k ${KEY_HEX_128} -b ${inBootstrapConf1}".split(" "))
+        then: "key is written to input bootstrap.conf"
+        assertBootstrapFilesAreEqual(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128, inBootstrapConf1, true)
+
+        when: "run with args: -p <password> -b <file>"
+        tool.run("-p ${PASSWORD} -b ${inBootstrapConf2}".split(" "))
+        then: "key derived from password is written to input bootstrap.conf"
+        PASSWORD_KEY_HEX == readKeyFromBootstrap(inBootstrapConf2)
+
+        when: "run with args: -k <key> -b <file> -B <file>"
+        tool.run("-k ${KEY_HEX_128} -b ${inBootstrapConf3} -B ${outBootstrapConf3}".split(" "))
+        then: "key is written to output bootstrap.conf"
+        assertBootstrapFilesAreEqual(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128, outBootstrapConf3, true)
+        and: "input bootstrap.conf is unchanged"
+        assertBootstrapFilesAreEqual(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT, inBootstrapConf3, true)
+
+        when: "run with args: -p <key> -b <file> -B <file>"
+        tool.run("-p ${PASSWORD} -b ${inBootstrapConf4} -B ${outBootstrapConf4}".split(" "))
+        then: "key derived from password is written to output bootstrap.conf"
+        PASSWORD_KEY_HEX == readKeyFromBootstrap(outBootstrapConf4)
+        and: "input bootstrap.conf is unchanged"
+        assertBootstrapFilesAreEqual(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT, inBootstrapConf4, true)
+
+        when: "run with args: -b <file> -B <file>"
+        tool.run("-b ${inBootstrapConf5} -B ${outBootstrapConf5}".split(" "))
+        then: "key from input file is copied to output file"
+        KEY_HEX_128 == readKeyFromBootstrap(outBootstrapConf5)
+        assertBootstrapFilesAreEqual(inBootstrapConf5, outBootstrapConf5, true)
+        and: "input bootstrap.conf is unchanged"
+        assertBootstrapFilesAreEqual(RESOURCE_REGISTRY_BOOTSTRAP_KEY_128, inBootstrapConf5, true)
+
+    }
+
+    def "encrypt unprotected nifi-registry.properties file"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED)
+        def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inRegistryProperties2 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED)
+        def outRegistryProperties2 = generateTmpFilePath()
+        def inBootstrapConf3 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inRegistryProperties3 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED)
+        def inRegistryProperties4 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED)
+
+        when: "run with args: -k <key> -b <file> -r <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -r ${inRegistryProperties1}".split(" "))
+        then: "properties file is protected in place"
+        assertNiFiRegistryUnprotectedPropertiesAreProtected(inRegistryProperties1)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf1)
+
+        when: "run with args: -k <key> -b <file> -r <file> -R <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf2} -r ${inRegistryProperties2} -R ${outRegistryProperties2}".split(" "))
+        then: "output properties file is protected"
+        assertNiFiRegistryUnprotectedPropertiesAreProtected(outRegistryProperties2)
+        and: "input properties file is unchanged"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED, inRegistryProperties2, true)
+        and: "key is written to output bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf2)
+
+        when: "run with args: -p <password> -b <file> -r <file>"
+        tool.run("-p ${PASSWORD} -b ${inBootstrapConf3} -r ${inRegistryProperties3}".split(" "))
+        then: "properties file is protected in place"
+        assertNiFiRegistryUnprotectedPropertiesAreProtected(inRegistryProperties3)
+        and: "key is written to input bootstrap.conf"
+        PASSWORD_KEY_HEX == readKeyFromBootstrap(inBootstrapConf3)
+
+        when: "run with args: -b <file_with_key> -r <file>"
+        tool.run("-b ${RESOURCE_REGISTRY_BOOTSTRAP_KEY_128} -r ${inRegistryProperties4}".split(" "))
+        then: "properties file is protected in place using key from bootstrap"
+        assertNiFiRegistryUnprotectedPropertiesAreProtected(inRegistryProperties4, PROTECTION_SCHEME_128)
+
+    }
+
+    def "encrypt nifi-registry.properties with no sensitive properties is a no-op"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_COMMENTED)
+        def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inRegistryProperties2 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_EMPTY)
+        def outRegistryProperties2 = generateTmpFilePath()
+
+        when: "run with args: -k <key> -b <file> -r <file_with_no_sensitive_props>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -r ${inRegistryProperties1}".split(" "))
+        then: "properties file is unchanged"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_COMMENTED, inRegistryProperties1, true)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf1)
+
+        when: "run with args: -k <key> -b <file> -r <file_with_empty_sensitive_props> -R <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf2} -r ${inRegistryProperties2} -R ${outRegistryProperties2}".split(" "))
+        then: "input properties file is unchanged and output properties file matches input"
+        assertPropertiesFilesAreEqual(RESOURCE_REGISTRY_PROPERTIES_EMPTY, inRegistryProperties2, true)
+        assertPropertiesFilesAreEqual(inRegistryProperties2, outRegistryProperties2, true)
+        and: "key is written to output bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf2)
+
+    }
+
+    def "encrypt unprotected authorizers.xml file"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inAuthorizers1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED)
+        def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inAuthorizers2 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED)
+        def outAuthorizers2 = generateTmpFilePath()
+
+        when: "run with args: -k <key> -b <file> -a <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -a ${inAuthorizers1}".split(" "))
+        then: "authorizers file is protected in place"
+        assertRegistryAuthorizersXmlIsProtected(inAuthorizers1)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf1)
+
+        when: "run with args: -k <key> -b <file> -a <file> -A <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf2} -a ${inAuthorizers2} -A ${outAuthorizers2}".split(" "))
+        then: "authorizers file is protected in place"
+        assertRegistryAuthorizersXmlIsProtected(outAuthorizers2)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf2)
+
+    }
+
+    def "encrypt authorizers.xml with no sensitive properties is a no-op"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inAuthorizersXml1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_COMMENTED)
+        def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inAuthorizersXml2 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_EMPTY)
+        def outAuthorizers2 = generateTmpFilePath()
+
+        when: "run with args: -k <key> -b <file> -a <file_with_no_sensitive_props>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -a ${inAuthorizersXml1}".split(" "))
+        then: "authorizers file is unchanged"
+        assertFilesAreEqual(RESOURCE_REGISTRY_AUTHORIZERS_COMMENTED, inAuthorizersXml1)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf1)
+
+        when: "run with args: -k <key> -b <file> -a <file_with_empty_sensitive_props> -A <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf2} -a ${inAuthorizersXml2} -A ${outAuthorizers2}".split(" "))
+        then: "input authorizers file is unchanged and output authorizers matches input"
+        assertFilesAreEqual(RESOURCE_REGISTRY_AUTHORIZERS_EMPTY, inAuthorizersXml2)
+        assertFilesAreEqual(inAuthorizersXml2, outAuthorizers2)
+        and: "key is written to output bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf2)
+
+    }
+
+    def "encrypt unprotected identity-providers.xml file"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED)
+        def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inIdentityProviders2 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED)
+        def outIdentityProviders2 = generateTmpFilePath()
+
+        when: "run with args: -k <key> -b <file> -i <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -i ${inIdentityProviders1}".split(" "))
+        then: "identity providers file is protected in place"
+        assertRegistryIdentityProvidersXmlIsProtected(inIdentityProviders1)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf1)
+
+        when: "run with args: -k <key> -b <file> -i <file> -I <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf2} -i ${inIdentityProviders2} -I ${outIdentityProviders2}".split(" "))
+        then: "identity providers file is protected in place"
+        assertRegistryIdentityProvidersXmlIsProtected(outIdentityProviders2)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf2)
+
+    }
+
+    def "encrypt identity-providers.xml with no sensitive properties is a no-op"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_COMMENTED)
+        def inBootstrapConf2 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inIdentityProviders2 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_EMPTY)
+        def outIdentityProviders2 = generateTmpFilePath()
+
+        when: "run with args: -k <key> -b <file> -i <file_with_no_sensitive_props>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -i ${inIdentityProviders1}".split(" "))
+        then: "identity providers file is unchanged"
+        assertFilesAreEqual(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_COMMENTED, inIdentityProviders1)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf1)
+
+        when: "run with args: -k <key> -b <file> -i <file_with_empty_sensitive_props> -I <file>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf2} -i ${inIdentityProviders2} -I ${outIdentityProviders2}".split(" "))
+        then: "identity providers file is unchanged and output identity providers matches input"
+        assertFilesAreEqual(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_EMPTY, inIdentityProviders2)
+        assertFilesAreEqual(inIdentityProviders2, outIdentityProviders2)
+        and: "key is written to output bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf2)
+
+    }
+
+    def "encrypt full configuration with properties, authorizers, and identity providers"() {
+
+        setup:
+        NiFiRegistryMode tool = new NiFiRegistryMode()
+        def inBootstrapConf1 = copyFileToTempFile(RESOURCE_REGISTRY_BOOTSTRAP_DEFAULT)
+        def inRegistryProperties1 = copyFileToTempFile(RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED)
+        def inAuthorizers1 = copyFileToTempFile(RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED)
+        def inIdentityProviders1 = copyFileToTempFile(RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED)
+
+        when: "run with args: -k <key> -b <file> -r <file> -a <file_with_no_sensitive_props> -i <file_with_no_sensitive_props>"
+        tool.run("-k ${KEY_HEX} -b ${inBootstrapConf1} -r ${inRegistryProperties1} -a ${inAuthorizers1} -i ${inIdentityProviders1}".split(" "))
+        then: "all files are protected"
+        assertNiFiRegistryUnprotectedPropertiesAreProtected(inRegistryProperties1)
+        assertRegistryAuthorizersXmlIsProtected(inAuthorizers1)
+        assertRegistryIdentityProvidersXmlIsProtected(inIdentityProviders1)
+        and: "key is written to input bootstrap.conf"
+        KEY_HEX == readKeyFromBootstrap(inBootstrapConf1)
+
+    }
+    
+    //-- Helper Methods
+
+    private static String readKeyFromBootstrap(String bootstrapPath) {
+        return BootstrapUtil.extractKeyFromBootstrapFile(bootstrapPath, BootstrapUtil.REGISTRY_BOOTSTRAP_KEY_PROPERTY)
+    }
+
+    private static boolean assertNiFiRegistryUnprotectedPropertiesAreProtected(
+            String pathToProtectedProperties,
+            String expectedProtectionScheme = PROTECTION_SCHEME) {
+        return assertPropertiesAreProtected(
+                RESOURCE_REGISTRY_PROPERTIES_POPULATED_UNPROTECTED,
+                pathToProtectedProperties,
+                RESOURCE_REGISTRY_PROPERTIES_SENSITIVE_PROPS,
+                expectedProtectionScheme)
+    }
+
+    static boolean assertRegistryAuthorizersXmlIsProtected(
+            String pathToProtectedXmlToVerify,
+            String expectedProtectionScheme = PROTECTION_SCHEME,
+            String expectedKey = KEY_HEX) {
+        return assertRegistryAuthorizersXmlIsProtected(
+                RESOURCE_REGISTRY_AUTHORIZERS_POPULATED_UNPROTECTED,
+                pathToProtectedXmlToVerify,
+                expectedProtectionScheme,
+                expectedKey)
+    }
+
+    static boolean assertRegistryIdentityProvidersXmlIsProtected(
+            String pathToProtectedXmlToVerify,
+            String expectedProtectionScheme = PROTECTION_SCHEME,
+            String expectedKey = KEY_HEX) {
+        return assertRegistryIdentityProvidersXmlIsProtected(
+                RESOURCE_REGISTRY_IDENTITY_PROVIDERS_POPULATED_UNPROTECTED,
+                pathToProtectedXmlToVerify,
+                expectedProtectionScheme,
+                expectedKey)
+    }
+
+
+}