You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/08/05 16:46:54 UTC

[GitHub] [nifi] thenatog opened a new pull request, #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

thenatog opened a new pull request, #6273:
URL: https://github.com/apache/nifi/pull/6273

   This is a draft PR of adding the property encryptor module. I would like input on the general approach, any issues with naming, or suggestions for improvement.
   
   At the moment this can be run by building and running:
   
   `java -cp "./target/nifi-property-encryptor-tool-1.18.0-SNAPSHOT-jar-with-dependencies.jar:~/.m2/repository/info/picocli/picocli/4.5.2/picocli-4.5.2.jar:~/.m2/repository/org/slf4j/slf4j-simple/1.7.36/slf4j-simple-1.7.36.jar:~/.m2/repository/org/slf4j/slf4j-api/1.7.36/slf4j-api-1.7.36.jar:../../nifi-commons/nifi-property-protection-factory/target/nifi-property-protection-factory-1.18.0-SNAPSHOT.jar"  org.apache.nifi.util.console.PropertyEncryptorCLI encrypt config /tmp/encrypt-me aSecurePassphrase AES_GCM`
   
   though it seems the references to maven repository may need to be absolute rather than using tilde. I have not yet determined how to build the picocli application to run more like a bash script but that is the end goal.
   
   This PR should encrypt configuration files with AES_GCM - other providers have not yet been added. Also still to add is encrypting the nifi.properties (which should be an additional commit to this PR) and encrypting the flow.xml.gz/flow.json.gz which might be a separate PR. Currently it is also writing out the encrypted files as temporary files, without overwriting the existing ones. I think I will instead add some changes to this PR to copy the old files to .bkup and rename the encrypted files. 
   
   
   <!-- 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. -->
   
   # Summary
   
   [NIFI-9953](https://issues.apache.org/jira/browse/NIFI-9953)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI-9953) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r1018358816


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/utils/SchemeCandidates.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.util.console.utils;
+
+import org.apache.nifi.properties.scheme.StandardProtectionSchemeResolver;
+
+import java.util.ArrayList;
+
+public class SchemeCandidates extends ArrayList<String> {
+    SchemeCandidates() {
+        super(new StandardProtectionSchemeResolver().getSupportedProtectionSchemes());

Review Comment:
   Instead of returning the internal enum names, we should return the values of `ProtectionScheme.getPath()`. Those are the values used in the configuration files, and this is a good opportunity to switch the public arguments.



##########
nifi-registry/nifi-registry-core/nifi-registry-properties-loader/pom.xml:
##########
@@ -54,5 +54,17 @@
             <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-property-protection-loader</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties-loader</artifactId>
+            <version>1.18.0-SNAPSHOT</version>

Review Comment:
   ```suggestion
               <version>1.19.0-SNAPSHOT</version>
   ```



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/PropertyEncryptorCommand.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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;
+
+import org.apache.nifi.encrypt.PropertyEncryptionMethod;
+import org.apache.nifi.encrypt.PropertyEncryptor;
+import org.apache.nifi.encrypt.PropertyEncryptorBuilder;
+import org.apache.nifi.flow.encryptor.StandardFlowEncryptor;
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.MutableApplicationProperties;
+import org.apache.nifi.properties.MutableBootstrapProperties;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
+import org.apache.nifi.properties.PropertiesLoader;
+import org.apache.nifi.properties.ProtectedPropertyContext;
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.registry.properties.NiFiRegistryPropertiesLoader;
+import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapPropertiesLoader;
+import org.apache.nifi.security.util.KeyDerivationFunction;
+import org.apache.nifi.security.util.crypto.SecureHasherFactory;
+import org.apache.nifi.serde.StandardPropertiesWriter;
+import org.apache.nifi.util.NiFiBootstrapPropertiesLoader;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.util.file.ConfigurationFileResolver;
+import org.apache.nifi.util.file.ConfigurationFileUtils;
+import org.apache.nifi.util.file.NiFiConfigurationFileResolver;
+import org.apache.nifi.util.file.NiFiFlowDefinitionFileResolver;
+import org.apache.nifi.util.file.NiFiRegistryConfigurationFileResolver;
+import org.apache.nifi.util.properties.NiFiRegistrySensitivePropertyResolver;
+import org.apache.nifi.util.properties.NiFiSensitivePropertyResolver;
+import org.apache.nifi.util.properties.SensitivePropertyResolver;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Properties;
+import java.util.zip.GZIPInputStream;
+
+public class PropertyEncryptorCommand {
+
+    private static final Logger logger = LoggerFactory.getLogger(PropertyEncryptorCommand.class);
+    private final AbstractBootstrapPropertiesLoader bootstrapLoader;
+    private final PropertiesLoader<ApplicationProperties> propertiesLoader;
+    private final ConfigurationFileResolver fileResolver;
+    private final SensitivePropertyResolver sensitivePropertyResolver;
+    private final List<File> configurationFiles;
+    private String inputHexKey;
+    private String outputHexKey;
+    private String inputSensitivePropertyKey;
+    private String outputSensitivePropertyKey;
+    private final Path confDirectory;
+    private final ApplicationProperties properties;
+    private final File applicationPropertiesFile;
+    private final Path outputDirectory;
+
+    public PropertyEncryptorCommand(final Path baseDirectory, final String passphrase) throws PropertyEncryptorException {
+        confDirectory = ConfigurationFileUtils.resolveAbsoluteConfDirectory(baseDirectory);
+        try {
+            bootstrapLoader = getBootstrapPropertiesLoader(confDirectory);
+            fileResolver = getConfigurationFileResolver(confDirectory);
+            applicationPropertiesFile = ConfigurationFileUtils.resolvePropertiesFile(confDirectory);
+            propertiesLoader = getPropertiesLoader(confDirectory);
+            properties = propertiesLoader.load(applicationPropertiesFile);
+            configurationFiles = fileResolver.resolveFilesFromApplicationProperties(properties);
+            sensitivePropertyResolver = getSensitivePropertyResolver(confDirectory);
+            inputHexKey = getExistingRootKey(confDirectory, passphrase);
+            outputHexKey = deriveOutputKey(passphrase);
+            outputDirectory = ConfigurationFileUtils.getOutputDirectory(baseDirectory);
+            logger.info("Output directory created at [{}]", outputDirectory.toAbsolutePath());
+        } catch (final Exception e) {
+            throw new PropertyEncryptorException("Failed to run property encryptor", e);
+        }
+    }
+
+    /**
+     * @param baseDirectory The base directory of a NiFi / NiFi Registry installation that should be encrypted
+     */
+    public void encryptXmlConfigurationFiles(final Path baseDirectory, final ProtectionScheme scheme) {
+        XmlEncryptor encryptor = getXmlEncryptor(scheme);
+        try {
+            encryptXmlConfigurationFiles(configurationFiles, encryptor);
+            logger.info("The Property Encryptor successfully encrypted configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath());
+        } catch (Exception e) {
+            logger.error("The Property Encryptor failed to encrypt configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath(), e);
+        }
+    }
+
+    public void encryptPropertiesFile(final ProtectionScheme scheme) throws IOException {
+        List<String> sensitivePropertyKeys = sensitivePropertyResolver.resolveSensitivePropertyKeys(properties);
+        final MutableApplicationProperties encryptedProperties = new MutableApplicationProperties(new Properties());
+        final SensitivePropertyProvider provider = StandardSensitivePropertyProviderFactory.withKey(inputHexKey).getProvider(scheme);
+
+        for (String key : sensitivePropertyKeys) {
+            if (properties.getProperty(key) != null) {
+                String encryptedValue = provider.protect(properties.getProperty(key), ProtectedPropertyContext.defaultContext(key));
+                encryptedProperties.setProperty(key, encryptedValue);
+            }
+        }
+
+        final File outputPropertiesFile = ConfigurationFileUtils.getOutputFile(outputDirectory, applicationPropertiesFile);
+        try (FileInputStream inputStream = new FileInputStream(applicationPropertiesFile);
+             FileOutputStream outputStream = new FileOutputStream(outputPropertiesFile)) {
+            new StandardPropertiesWriter().writePropertiesFile(inputStream, outputStream, encryptedProperties);
+        }
+    }
+
+    private void encryptXmlConfigurationFiles(final List<File> configurationFiles, final XmlEncryptor encryptor) {
+        for (final File configurationFile : configurationFiles) {
+            File outputFile = ConfigurationFileUtils.getOutputFile(outputDirectory, configurationFile);
+            try (InputStream inputStream = new FileInputStream(configurationFile);
+                FileOutputStream outputStream = new FileOutputStream(outputFile)) {
+                encryptor.encrypt(inputStream, outputStream);
+                logger.info("Successfully encrypted file at [{}], and output to [{}]", configurationFile.getAbsolutePath(), outputFile.getAbsolutePath());
+            } catch (Exception e) {
+                throw new PropertyEncryptorException(String.format("Failed to encrypt configuration file: [%s]", configurationFile.getAbsolutePath()), e);
+            }
+        }
+    }
+
+    public void migrateConfigurationFiles(final File baseDirectory) {
+        logger.info("Not yet implemented.");
+    }
+
+    public void encryptFlowDefinition(final PropertyEncryptionMethod algorithm, final String passphrase) {
+        final StandardFlowEncryptor flowEncryptor = new StandardFlowEncryptor();
+        final String existingSensitivePropertyKey = getSensitivePropertyKey();
+        final String existingEncryptionAlgorithm = getSensitivePropertyEncryptionAlgorithm();
+
+        PropertyEncryptor inputEncryptor = new PropertyEncryptorBuilder(existingSensitivePropertyKey).setAlgorithm(existingEncryptionAlgorithm).build();
+        PropertyEncryptor outputEncryptor = new PropertyEncryptorBuilder(passphrase).setAlgorithm(algorithm).build();
+
+        if (properties instanceof NiFiProperties) {
+            List<File> flowDefinitions = new NiFiFlowDefinitionFileResolver().resolveFilesFromApplicationProperties((NiFiProperties) properties);
+            for (File flow : flowDefinitions) {
+                final File absoluteFlowDefinition = ConfigurationFileUtils.getAbsoluteFile(confDirectory.toFile(), flow);
+                try {
+                    File output = ConfigurationFileUtils.getOutputFile(outputDirectory, absoluteFlowDefinition);
+                    try (InputStream inputStream = new GZIPInputStream(new FileInputStream(absoluteFlowDefinition));
+                         FileOutputStream outputStream = new FileOutputStream(output)) {
+                        flowEncryptor.processFlow(inputStream, outputStream, inputEncryptor, outputEncryptor);
+                    }
+                } catch (IOException e) {
+                    logger.error("Failed to encrypt flow definition file: [{}]", flow.getAbsolutePath(), e);
+                }
+            }
+        }
+    }
+
+    public void outputSensitiveProperties(final PropertyEncryptionMethod algorithm, final String sensitivePropertyKey) {
+        final File outputPropertiesFile = ConfigurationFileUtils.getOutputFile(outputDirectory, applicationPropertiesFile);
+        final MutableApplicationProperties updatedProperties = new MutableApplicationProperties(new Properties());
+        updatedProperties.setProperty(NiFiProperties.SENSITIVE_PROPS_KEY, sensitivePropertyKey);
+        updatedProperties.setProperty(NiFiProperties.SENSITIVE_PROPS_ALGORITHM, algorithm.name());
+
+        try {
+            try (InputStream inputStream = new FileInputStream(applicationPropertiesFile);
+                 FileOutputStream outputStream = new FileOutputStream(outputPropertiesFile)) {
+                new StandardPropertiesWriter().writePropertiesFile(inputStream, outputStream, updatedProperties);
+            }
+            logger.info("Output sensitive properties key to {}", applicationPropertiesFile.getAbsolutePath());
+        } catch (IOException e) {
+            logger.error("Failed to output sensitive property keys after encrypting flow", e);
+        }
+    }
+
+    public void outputKeyToBootstrap() throws IOException {
+        final File bootstrapFile = bootstrapLoader.getBootstrapFileWithinConfDirectory(confDirectory);
+        final File outputBootstrapFile = ConfigurationFileUtils.getOutputFile(outputDirectory, bootstrapFile);
+        final MutableBootstrapProperties bootstrapProperties = bootstrapLoader.loadMutableBootstrapProperties(bootstrapFile.getPath());
+        bootstrapProperties.setProperty(BootstrapProperties.BootstrapPropertyKey.SENSITIVE_KEY.getKey(), outputHexKey);
+        try (InputStream inputStream = new FileInputStream(bootstrapFile);
+             FileOutputStream outputStream = new FileOutputStream(outputBootstrapFile)) {
+            new StandardPropertiesWriter().writePropertiesFile(inputStream, outputStream, bootstrapProperties);
+            logger.info("Output the bootstrap key to {}", outputBootstrapFile);
+        }
+    }
+
+    private XmlEncryptor getXmlEncryptor(final ProtectionScheme scheme) {
+        final SensitivePropertyProviderFactory providerFactory = StandardSensitivePropertyProviderFactory.withKey(inputHexKey);
+        return new XmlEncryptor(providerFactory, scheme);
+    }
+
+    private XmlDecryptor getXmlDecryptor(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        return new XmlDecryptor(providerFactory, scheme);
+    }
+
+    private String getExistingRootKey(final Path confDirectory, final String passphrase) {

Review Comment:
   It looks like this is supposed to fall back to the supplied passphrase, but the `passphrase` argument is not specified.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r960075378


##########
nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/PropertyProtectionScheme.java:
##########
@@ -19,7 +19,7 @@
 /**
  * Property Protection Schemes supported as arguments for encryption commands should not have direct references
  */
-enum PropertyProtectionScheme implements ProtectionScheme {
+public enum PropertyProtectionScheme implements ProtectionScheme {

Review Comment:
   I added a SchemeCandidates class which utilises the StandardProtectionSchemeResolver instead of directly using the PropertyProtectionScheme enum. Does this resolve the issue at hand?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory closed pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified
URL: https://github.com/apache/nifi/pull/6273


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r959785072


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {
+
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+    protected static final String PARENT_IDENTIFIER = "identifier";
+    protected static final String PROPERTY_ELEMENT = "property";
+
+    protected final SensitivePropertyProvider cryptoProvider;
+    protected SensitivePropertyProviderFactory providerFactory;
+
+    public XmlCryptoParser(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        this.providerFactory = providerFactory;
+        cryptoProvider = providerFactory.getProvider(scheme);
+    }
+
+    protected void cryptographicXmlOperation(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        final XMLOutputFactory factory = XMLOutputFactory.newInstance();
+        factory.setProperty("com.ctc.wstx.outputValidateStructure", false);

Review Comment:
   There was some reason I set this, and it looked like woodstox was being used which led to this property. I'll re-check why and if it can be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r955317722


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {

Review Comment:
   Does this mean an interface that the XmlCryptoParser would implement, or an interface that the XmlDecryptor/XmlEncryptor classes would implement? My main goal here was to reuse as much code as possible as initially I had very similar code between an encryptor and decryptor class which was unnecessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r973392467


##########
nifi-toolkit/nifi-property-encryptor-tool/pom.xml:
##########
@@ -0,0 +1,151 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-toolkit</artifactId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-property-encryptor-tool</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>info.picocli</groupId>
+            <artifactId>picocli</artifactId>
+            <version>4.6.3</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-property-protection-factory</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties-loader</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties-loader</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>${org.slf4j.version}</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <version>${org.slf4j.version}</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-xml-processing</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-property-encryptor</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-flow-encryptor</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <archive>
+                        <manifest>
+                            <mainClass>org.apache.nifi.util.console.PropertyEncryptorCLI</mainClass>
+                        </manifest>
+                    </archive>
+                    <descriptorRefs>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                    </descriptorRefs>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>appassembler-maven-plugin</artifactId>
+                <version>2.1.0</version>
+                <configuration>
+                    <programs>
+                        <program>
+                            <mainClass>org.apache.nifi.util.console.PropertyEncryptorCLI</mainClass>
+                            <id>property-encryptor</id>
+                        </program>
+                    </programs>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <annotationProcessorPaths>
+                        <path>
+                            <groupId>info.picocli</groupId>
+                            <artifactId>picocli-codegen</artifactId>
+                            <version>4.6.3</version>
+                        </path>
+                    </annotationProcessorPaths>
+                    <compilerArgs>
+                        <arg>-Aproject=${project.groupId}/${project.artifactId}</arg>
+                    </compilerArgs>
+                </configuration>
+            </plugin>

Review Comment:
   It looks like these two Maven plugin configurations could be removed for now and revisited in a subsequent issue.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r968547718


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/serde/StandardPropertiesWriter.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.serde;
+
+import org.apache.nifi.properties.ReadableProperties;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class StandardPropertiesWriter implements PropertiesWriter {
+
+    private static final String PROPERTY_FORMAT = "(%s)=(%s)";
+
+    public void writePropertiesFile(final InputStream inputStream, final OutputStream outputStream, final ReadableProperties properties) throws IOException {
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream));
+             BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(outputStream))) {
+            String line;
+            while ((line = reader.readLine()) != null) {
+                Set<String> keys = properties.getPropertyKeys();
+                for (final String key : keys) {
+                    Pattern regex = Pattern.compile(PROPERTY_FORMAT);
+                    Matcher matcher = regex.matcher(line);
+                    if (matcher.matches() && matcher.group(0).equals(key)) {
+                        line = String.format(PROPERTY_FORMAT, key, properties.getProperty(key));

Review Comment:
   The pattern string need to include the current `key` in order for the match to work, and it looks like a different format string will be necessary:
   ```suggestion
                       Pattern regex = Pattern.compile(String.format("^%s=.*$", key));
                       Matcher matcher = regex.matcher(line);
                       if (matcher.matches()) {
                           line = String.format(PROPERTY_FORMAT, key, properties.getProperty(key));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on PR #6273:
URL: https://github.com/apache/nifi/pull/6273#issuecomment-1240837848

   I've updated the PR with most of the requested changes, let me know if further changes are necessary


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r941394985


##########
nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/PropertyProtectionScheme.java:
##########
@@ -19,7 +19,7 @@
 /**
  * Property Protection Schemes supported as arguments for encryption commands should not have direct references
  */
-enum PropertyProtectionScheme implements ProtectionScheme {
+public enum PropertyProtectionScheme implements ProtectionScheme {

Review Comment:
   This enum was intentionally package-private to encapsulate resolution of the scheme. As indicated in the enum values, there are different ways of referring to the scheme. With the introduction of a new command, this seems like a good opportunity to change the argument contract to use the scheme values, instead of the enum names.



##########
nifi-toolkit/nifi-property-encryptor-tool/pom.xml:
##########
@@ -0,0 +1,129 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-property-encryptor-tool</artifactId>
+
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>info.picocli</groupId>
+            <artifactId>picocli</artifactId>
+            <version>4.6.3</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-property-protection-factory</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties-loader</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties-loader</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>${org.slf4j.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <version>${org.slf4j.version}</version>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <archive>
+                        <manifest>
+                            <mainClass>org.apache.nifi.util.console.PropertyEncryptorCLI</mainClass>
+                        </manifest>
+                    </archive>
+                    <descriptorRefs>
+                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                    </descriptorRefs>
+                </configuration>
+                <executions>
+                    <execution>
+                        <id>make-assembly</id>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>single</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>appassembler-maven-plugin</artifactId>
+                <version>2.1.0</version>
+                <configuration>
+                    <programs>
+                        <program>
+                            <mainClass>org.apache.nifi.util.console.PropertyEncryptorCLI</mainClass>
+                            <id>property-encryptor</id>
+                        </program>
+                    </programs>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <!-- annotationProcessorPaths requires maven-compiler-plugin version 3.5 or higher -->

Review Comment:
   Is this comment necessary? The Maven compiler plugin version should be set in the root configuration.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/PropertyEncryptorMain.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.MutableBootstrapProperties;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapPropertiesLoader;
+import org.apache.nifi.serde.PropertiesWriter;
+import org.apache.nifi.util.NiFiBootstrapPropertiesLoader;
+import org.apache.nifi.util.crypto.CryptographyUtils;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.apache.nifi.util.file.ConfigurationFileResolver;
+import org.apache.nifi.util.file.FileUtilities;
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
+import org.apache.nifi.properties.PropertiesLoader;
+import org.apache.nifi.registry.properties.NiFiRegistryPropertiesLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+
+public class PropertyEncryptorMain {
+
+    private static final Logger logger = LoggerFactory.getLogger(PropertyEncryptorMain.class);
+    private final AbstractBootstrapPropertiesLoader bootstrapLoader;
+    private final PropertiesLoader<ApplicationProperties> propertiesLoader;
+    private final ConfigurationFileResolver fileResolver;
+    private final List<File> configurationFiles;
+    private String hexKey;
+    private final Path confDirectory;
+
+    public PropertyEncryptorMain(final Path baseDirectory, final String passphrase) throws PropertyEncryptorException {
+        confDirectory = FileUtilities.resolveAbsoluteConfDirectory(baseDirectory);
+        try {
+            bootstrapLoader = getBootstrapPropertiesLoader(confDirectory);
+            fileResolver = new ConfigurationFileResolver(confDirectory);
+            final File applicationProperties = FileUtilities.resolvePropertiesFile(confDirectory);
+            propertiesLoader = getPropertiesLoader(confDirectory);
+            configurationFiles = fileResolver.resolveConfigurationFilesFromApplicationProperties(propertiesLoader.load(applicationProperties));
+            hexKey = getKeyHex(confDirectory, passphrase);
+        } catch (final Exception e) {
+            throw new PropertyEncryptorException("Failed to run property encryptor", e);
+        }
+    }
+
+    private AbstractBootstrapPropertiesLoader getBootstrapPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryBootstrapPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiBootstrapPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized bootstrap.conf file");
+        }
+    }
+
+    private PropertiesLoader<ApplicationProperties> getPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized .properties file");
+        }
+    }
+
+    /**
+     * @param baseDirectory The base directory of a NiFi / NiFi Registry installation that should be encrypted
+     */
+    public int encryptConfigurationFiles(final Path baseDirectory, final PropertyProtectionScheme scheme) {
+        XmlEncryptor encryptor = getXmlEncryptor(scheme);
+        try {
+            encryptConfigurationFiles(configurationFiles, encryptor);
+            outputKeyToBootstrap();
+            logger.info("The Property Encryptor successfully encrypted configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath());
+            return 0;
+        } catch (Exception e) {
+            logger.error("The Property Encryptor failed to encrypt configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath(), e);
+            return 1;
+        }
+    }
+
+    private void outputKeyToBootstrap() throws IOException {
+        final File bootstrapFile = bootstrapLoader.getBootstrapFileWithinConfDirectory(confDirectory);
+        File tempBootstrapFile = FileUtilities.getTemporaryOutputFile("tmp", bootstrapFile);

Review Comment:
   The reference to `tmp` should be promoted to a static variable and reused.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/ConfigSubcommand.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.util.console;
+
+import org.apache.nifi.PropertyEncryptorMain;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.util.console.utils.BaseCommandParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+@CommandLine.Command(name = "config",
+        description = "Operate on application configs",
+        usageHelpWidth=140
+)
+class ConfigSubcommand extends BaseCommandParameters implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConfigSubcommand.class);
+
+    @CommandLine.ParentCommand
+    private DefaultCLIOptions parent;
+
+    @CommandLine.Parameters(description="The encryption scheme to use, from one of the following schemes: [@|bold ${COMPLETION-CANDIDATES}|@]")
+    PropertyProtectionScheme scheme;

Review Comment:
   As mentioned on the enum, this is a good opportunity to revisit the supported arguments. Right now, the supported arguments are different than the values used to reference encrypted properties, so it would be helpful to have a unified approach.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/PropertyEncryptorTranslateForCLI.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.util.console;
+
+import picocli.CommandLine;
+
+@CommandLine.Command(name = "translate-for-nifi-cli", description = "@|bold,fg(blue) Translates|@ the nifi.properties file to a format suitable for use with the NiFi CLI tool")
+class PropertyEncryptorTranslateForCLI implements Runnable {

Review Comment:
   Are there applicable NiFi CLI commands for encryption and decryption? It seems like this may be unnecessary.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/crypto/CryptographyUtils.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.util.crypto;
+
+import org.apache.commons.codec.binary.Hex;
+import org.bouncycastle.crypto.generators.SCrypt;
+
+import javax.crypto.Cipher;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.List;
+
+public class CryptographyUtils {
+
+    private static final String NIFI_SCRYPT_SALT = "NIFI_SCRYPT_SALT";
+    private static final int DEFAULT_MIN_PASSPHRASE_LENGTH = 12;
+
+    // Strong parameters as of 12 Aug 2016
+    private static final int SCRYPT_N = (int) Math.pow(2, 16);
+    private static final int SCRYPT_R = 8;
+    private static final int SCRYPT_P = 1;

Review Comment:
   It seems the like scrypt details should be abstracted in some kind of interface, perhaps a `SaltGenerator`?



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {

Review Comment:
   Similar to other components, defining an interface would provide a clear contract, as opposed to an abstract base class.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/test/java/org/apache/nifi/encryptor/XmlEncryptorTest.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.encryptor;
+
+import org.apache.nifi.util.crypto.CryptographyUtils;
+import org.apache.nifi.util.file.FileUtilities;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.properties.scheme.StandardProtectionScheme;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.junit.jupiter.api.Test;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URISyntaxException;
+import java.nio.file.Files;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+class XmlEncryptorTest {
+
+    private static final String KEY_HEX_128 = "0123456789ABCDEFFEDCBA9876543210";
+    private static final String KEY_HEX_256 = KEY_HEX_128 + KEY_HEX_128;
+    static final ProtectionScheme DEFAULT_PROTECTION_SCHEME = new StandardProtectionScheme("aes/gcm");
+    public static final String KEY_HEX = CryptographyUtils.isUnlimitedStrengthCryptoAvailable() ? KEY_HEX_256 : KEY_HEX_128;

Review Comment:
   This can be refactored to just use AES-256 since Java 8 Update 171 and later do not limit key sizes.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlDecryptor.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Objects;
+
+public class XmlDecryptor extends XmlCryptoParser {
+
+    protected static final String ENCRYPTION_NONE = "none";
+    protected static final String PROPERTY_ELEMENT = "property";
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+
+    public XmlDecryptor(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        super(providerFactory, scheme);
+        this.providerFactory = providerFactory;
+    }
+
+    public void decrypt(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        cryptographicXmlOperation(encryptedXmlContent, decryptedOutputStream);
+    }
+
+    @Override
+    protected StartElement updateStartElementEncryptionAttribute(XMLEvent xmlEvent) {
+        return convertToDecryptedElement(xmlEvent);
+    }
+
+    @Override
+    protected Characters cryptoOperationOnCharacters(XMLEvent xmlEvent, String groupIdentifier, final String propertyName) {
+        return decryptElementCharacters(xmlEvent, groupIdentifier, propertyName);
+    }
+
+    /**
+     * Decrypt the XMLEvent element characters/value, which should contain an encrypted value
+     * @param xmlEvent The encrypted Characters event to be decrypted
+     * @return The decrypted Characters event
+     */
+    private Characters decryptElementCharacters(final XMLEvent xmlEvent, final String groupIdentifier, final String propertyName) {
+        final XMLEventFactory eventFactory = XMLEventFactory.newInstance();
+        final String encryptedCharacters = xmlEvent.asCharacters().getData().trim();
+        String decryptedCharacters = cryptoProvider.unprotect(encryptedCharacters, providerFactory.getPropertyContext(groupIdentifier, propertyName));
+        return eventFactory.createCharacters(decryptedCharacters);
+    }
+
+    /**
+     * Takes a StartElement and updates the 'encrypted' attribute to empty string to remove the encryption method/scheme
+     * @param xmlEvent The opening/start XMLEvent for an encrypted property
+     * @return The updated element to be written to XML file
+     */
+    private StartElement convertToDecryptedElement(final XMLEvent xmlEvent) {
+        if (isEncryptedElement(xmlEvent)) {
+            return updateElementAttribute(xmlEvent, ENCRYPTION_ATTRIBUTE_NAME, ENCRYPTION_NONE);
+        } else {
+            throw new XmlCryptoException(String.format("Failed to update the element's [%s] attribute when decrypting the element value", ENCRYPTION_ATTRIBUTE_NAME));
+        }
+    }
+
+    public boolean isEncryptedElement(final XMLEvent xmlEvent) {
+        return xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PROPERTY_ELEMENT)
+                && elementHasEncryptionAttribute(xmlEvent.asStartElement());
+    }
+
+    private boolean elementHasEncryptionAttribute(final StartElement xmlEvent) {
+        return xmlElementHasAttribute(xmlEvent, ENCRYPTION_ATTRIBUTE_NAME);
+    }
+
+    private boolean xmlElementHasAttribute(final StartElement xmlEvent, final String attributeName) {
+        return !Objects.isNull(xmlEvent.getAttributeByName(new QName(attributeName)));
+    }
+
+    public XMLEventReader getXMLReader(final InputStream fileStream) throws XMLStreamException {
+        XMLInputFactory xmlInputFactory = XMLInputFactory.newFactory();
+        xmlInputFactory.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+        xmlInputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+
+        return xmlInputFactory.createXMLEventReader(fileStream);
+    }

Review Comment:
   This should be refactored to use components from `nifi-xml-processing`.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/serde/PropertiesWriter.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.serde;
+
+import org.apache.nifi.properties.ReadableProperties;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Set;
+
+public class PropertiesWriter {

Review Comment:
   Recommend declaring an interface and creating a standard implementation, even if it is just one method.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/PropertyEncryptorMain.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.MutableBootstrapProperties;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapPropertiesLoader;
+import org.apache.nifi.serde.PropertiesWriter;
+import org.apache.nifi.util.NiFiBootstrapPropertiesLoader;
+import org.apache.nifi.util.crypto.CryptographyUtils;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.apache.nifi.util.file.ConfigurationFileResolver;
+import org.apache.nifi.util.file.FileUtilities;
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
+import org.apache.nifi.properties.PropertiesLoader;
+import org.apache.nifi.registry.properties.NiFiRegistryPropertiesLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+
+public class PropertyEncryptorMain {
+
+    private static final Logger logger = LoggerFactory.getLogger(PropertyEncryptorMain.class);
+    private final AbstractBootstrapPropertiesLoader bootstrapLoader;
+    private final PropertiesLoader<ApplicationProperties> propertiesLoader;
+    private final ConfigurationFileResolver fileResolver;
+    private final List<File> configurationFiles;
+    private String hexKey;
+    private final Path confDirectory;
+
+    public PropertyEncryptorMain(final Path baseDirectory, final String passphrase) throws PropertyEncryptorException {
+        confDirectory = FileUtilities.resolveAbsoluteConfDirectory(baseDirectory);
+        try {
+            bootstrapLoader = getBootstrapPropertiesLoader(confDirectory);
+            fileResolver = new ConfigurationFileResolver(confDirectory);
+            final File applicationProperties = FileUtilities.resolvePropertiesFile(confDirectory);
+            propertiesLoader = getPropertiesLoader(confDirectory);
+            configurationFiles = fileResolver.resolveConfigurationFilesFromApplicationProperties(propertiesLoader.load(applicationProperties));
+            hexKey = getKeyHex(confDirectory, passphrase);
+        } catch (final Exception e) {
+            throw new PropertyEncryptorException("Failed to run property encryptor", e);
+        }
+    }
+
+    private AbstractBootstrapPropertiesLoader getBootstrapPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryBootstrapPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiBootstrapPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized bootstrap.conf file");
+        }
+    }
+
+    private PropertiesLoader<ApplicationProperties> getPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized .properties file");

Review Comment:
   It would be helpful to include the `baseDirectory` in the message.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/PropertyEncryptorMain.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.MutableBootstrapProperties;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapPropertiesLoader;
+import org.apache.nifi.serde.PropertiesWriter;
+import org.apache.nifi.util.NiFiBootstrapPropertiesLoader;
+import org.apache.nifi.util.crypto.CryptographyUtils;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.apache.nifi.util.file.ConfigurationFileResolver;
+import org.apache.nifi.util.file.FileUtilities;
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
+import org.apache.nifi.properties.PropertiesLoader;
+import org.apache.nifi.registry.properties.NiFiRegistryPropertiesLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+
+public class PropertyEncryptorMain {
+
+    private static final Logger logger = LoggerFactory.getLogger(PropertyEncryptorMain.class);
+    private final AbstractBootstrapPropertiesLoader bootstrapLoader;
+    private final PropertiesLoader<ApplicationProperties> propertiesLoader;
+    private final ConfigurationFileResolver fileResolver;
+    private final List<File> configurationFiles;
+    private String hexKey;
+    private final Path confDirectory;
+
+    public PropertyEncryptorMain(final Path baseDirectory, final String passphrase) throws PropertyEncryptorException {
+        confDirectory = FileUtilities.resolveAbsoluteConfDirectory(baseDirectory);
+        try {
+            bootstrapLoader = getBootstrapPropertiesLoader(confDirectory);
+            fileResolver = new ConfigurationFileResolver(confDirectory);
+            final File applicationProperties = FileUtilities.resolvePropertiesFile(confDirectory);
+            propertiesLoader = getPropertiesLoader(confDirectory);
+            configurationFiles = fileResolver.resolveConfigurationFilesFromApplicationProperties(propertiesLoader.load(applicationProperties));
+            hexKey = getKeyHex(confDirectory, passphrase);
+        } catch (final Exception e) {
+            throw new PropertyEncryptorException("Failed to run property encryptor", e);
+        }
+    }
+
+    private AbstractBootstrapPropertiesLoader getBootstrapPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryBootstrapPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiBootstrapPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized bootstrap.conf file");
+        }
+    }
+
+    private PropertiesLoader<ApplicationProperties> getPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized .properties file");
+        }
+    }
+
+    /**
+     * @param baseDirectory The base directory of a NiFi / NiFi Registry installation that should be encrypted
+     */
+    public int encryptConfigurationFiles(final Path baseDirectory, final PropertyProtectionScheme scheme) {
+        XmlEncryptor encryptor = getXmlEncryptor(scheme);
+        try {
+            encryptConfigurationFiles(configurationFiles, encryptor);
+            outputKeyToBootstrap();
+            logger.info("The Property Encryptor successfully encrypted configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath());
+            return 0;
+        } catch (Exception e) {
+            logger.error("The Property Encryptor failed to encrypt configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath(), e);
+            return 1;
+        }
+    }
+
+    private void outputKeyToBootstrap() throws IOException {
+        final File bootstrapFile = bootstrapLoader.getBootstrapFileWithinConfDirectory(confDirectory);
+        File tempBootstrapFile = FileUtilities.getTemporaryOutputFile("tmp", bootstrapFile);
+        final MutableBootstrapProperties bootstrapProperties = bootstrapLoader.loadMutableBootstrapProperties(bootstrapFile.getPath());
+        bootstrapProperties.setProperty(BootstrapProperties.BootstrapPropertyKey.SENSITIVE_KEY.getKey(), hexKey);
+        PropertiesWriter.writePropertiesFile(new FileInputStream(bootstrapFile), new FileOutputStream(tempBootstrapFile), bootstrapProperties);
+        logger.info("Output the bootstrap key to {}", tempBootstrapFile);
+    }
+
+    private int encryptConfigurationFiles(final List<File> configurationFiles, final XmlEncryptor encryptor) throws IOException {
+        for (final File configurationFile : configurationFiles) {
+            File temp = FileUtilities.getTemporaryOutputFile("tmp", configurationFile);
+            try (InputStream inputStream = new FileInputStream(configurationFile);
+                FileOutputStream outputStream = new FileOutputStream(temp)) {
+                encryptor.encrypt(inputStream, outputStream);
+                logger.info("Successfully encrypted [{}]", configurationFile.getAbsolutePath());
+            } catch (Exception e) {
+                throw new PropertyEncryptorException(String.format("Failed to encrypt configuration file: [%s]", configurationFile.getAbsolutePath()), e);
+            }
+
+            //Files.copy(temp.toPath(), configurationFile.toPath());
+        }
+        return 0;
+    }
+
+    public int migrateConfigurationFiles(final File baseDirectory) {
+        logger.info("Not yet implemented.");
+        return 0;
+    }
+
+    public int encryptFlowDefinition(final File baseDirectory) {
+        logger.info("Not yet implemented.");
+        return 0;
+    }
+
+    private XmlEncryptor getXmlEncryptor(final PropertyProtectionScheme scheme) {
+        final SensitivePropertyProviderFactory providerFactory = StandardSensitivePropertyProviderFactory.withKey(hexKey);
+        return new XmlEncryptor(providerFactory, scheme);
+    }
+
+    private XmlDecryptor getXmlDecryptor(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        return new XmlDecryptor(providerFactory, scheme);
+    }
+
+    private String getKeyHex(final Path confDirectory, final String passphrase) {

Review Comment:
   This is a good opportunity to name this method more clearly than the current tool. Perhaps `getEncodedRootKey`?



##########
nifi-toolkit/nifi-property-encryptor-tool/pom.xml:
##########
@@ -0,0 +1,129 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>nifi</artifactId>
+        <groupId>org.apache.nifi</groupId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-property-encryptor-tool</artifactId>
+
+    <properties>
+        <maven.compiler.source>8</maven.compiler.source>
+        <maven.compiler.target>8</maven.compiler.target>
+    </properties>

Review Comment:
   These properties are not necessary and should be removed.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/crypto/CryptographyUtils.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.util.crypto;
+
+import org.apache.commons.codec.binary.Hex;
+import org.bouncycastle.crypto.generators.SCrypt;
+
+import javax.crypto.Cipher;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.List;
+
+public class CryptographyUtils {
+
+    private static final String NIFI_SCRYPT_SALT = "NIFI_SCRYPT_SALT";
+    private static final int DEFAULT_MIN_PASSPHRASE_LENGTH = 12;
+
+    // Strong parameters as of 12 Aug 2016
+    private static final int SCRYPT_N = (int) Math.pow(2, 16);
+    private static final int SCRYPT_R = 8;
+    private static final int SCRYPT_P = 1;
+
+    public static String deriveKeyFromPassphrase(final String passphrase) throws KeyException, NoSuchAlgorithmException {
+        return deriveKeyFromPassphrase(passphrase, DEFAULT_MIN_PASSPHRASE_LENGTH);
+    }
+
+    public static boolean isUnlimitedStrengthCryptoAvailable() {
+        try {
+            return Cipher.getMaxAllowedKeyLength("AES") > 128;
+        } catch (NoSuchAlgorithmException e) {
+            return false;
+        }
+    }

Review Comment:
   This method is no longer necessary for versions of Java 8 after Update 171, so it can be removed.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/ConfigSubcommand.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.util.console;
+
+import org.apache.nifi.PropertyEncryptorMain;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.util.console.utils.BaseCommandParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+@CommandLine.Command(name = "config",
+        description = "Operate on application configs",
+        usageHelpWidth=140
+)
+class ConfigSubcommand extends BaseCommandParameters implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConfigSubcommand.class);
+
+    @CommandLine.ParentCommand
+    private DefaultCLIOptions parent;
+
+    @CommandLine.Parameters(description="The encryption scheme to use, from one of the following schemes: [@|bold ${COMPLETION-CANDIDATES}|@]")
+    PropertyProtectionScheme scheme;
+
+    @Override
+    public void run() {
+        final PropertyEncryptorMain propertyEncryptorMain = new PropertyEncryptorMain(baseDirectory, passphrase);

Review Comment:
   The instantiation and invocation of `PropertyEncryptorMain` calls into question the approach. Instead of having that `Main` class, it seems like more of the methods should be implemented in this class, or broken into their component parts.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/PropertyEncryptorMain.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.MutableBootstrapProperties;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapPropertiesLoader;
+import org.apache.nifi.serde.PropertiesWriter;
+import org.apache.nifi.util.NiFiBootstrapPropertiesLoader;
+import org.apache.nifi.util.crypto.CryptographyUtils;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.apache.nifi.util.file.ConfigurationFileResolver;
+import org.apache.nifi.util.file.FileUtilities;
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
+import org.apache.nifi.properties.PropertiesLoader;
+import org.apache.nifi.registry.properties.NiFiRegistryPropertiesLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+
+public class PropertyEncryptorMain {

Review Comment:
   As a general convention, recommend naming this `PropertyEncryptorCommand`.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/crypto/CryptographyUtils.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.util.crypto;
+
+import org.apache.commons.codec.binary.Hex;
+import org.bouncycastle.crypto.generators.SCrypt;
+
+import javax.crypto.Cipher;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.List;
+
+public class CryptographyUtils {

Review Comment:
   Rather than reintroducing a generalized utility class, it would be better to define one or more interface for specific purposes. In this case, a method for returning a derived key seems like the right interface contract.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/PropertyEncryptorMain.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.MutableBootstrapProperties;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapPropertiesLoader;
+import org.apache.nifi.serde.PropertiesWriter;
+import org.apache.nifi.util.NiFiBootstrapPropertiesLoader;
+import org.apache.nifi.util.crypto.CryptographyUtils;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.apache.nifi.util.file.ConfigurationFileResolver;
+import org.apache.nifi.util.file.FileUtilities;
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
+import org.apache.nifi.properties.PropertiesLoader;
+import org.apache.nifi.registry.properties.NiFiRegistryPropertiesLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+
+public class PropertyEncryptorMain {
+
+    private static final Logger logger = LoggerFactory.getLogger(PropertyEncryptorMain.class);
+    private final AbstractBootstrapPropertiesLoader bootstrapLoader;
+    private final PropertiesLoader<ApplicationProperties> propertiesLoader;
+    private final ConfigurationFileResolver fileResolver;
+    private final List<File> configurationFiles;
+    private String hexKey;
+    private final Path confDirectory;
+
+    public PropertyEncryptorMain(final Path baseDirectory, final String passphrase) throws PropertyEncryptorException {
+        confDirectory = FileUtilities.resolveAbsoluteConfDirectory(baseDirectory);
+        try {
+            bootstrapLoader = getBootstrapPropertiesLoader(confDirectory);
+            fileResolver = new ConfigurationFileResolver(confDirectory);
+            final File applicationProperties = FileUtilities.resolvePropertiesFile(confDirectory);
+            propertiesLoader = getPropertiesLoader(confDirectory);
+            configurationFiles = fileResolver.resolveConfigurationFilesFromApplicationProperties(propertiesLoader.load(applicationProperties));
+            hexKey = getKeyHex(confDirectory, passphrase);
+        } catch (final Exception e) {
+            throw new PropertyEncryptorException("Failed to run property encryptor", e);
+        }
+    }
+
+    private AbstractBootstrapPropertiesLoader getBootstrapPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryBootstrapPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiBootstrapPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized bootstrap.conf file");
+        }
+    }
+
+    private PropertiesLoader<ApplicationProperties> getPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized .properties file");
+        }
+    }
+
+    /**
+     * @param baseDirectory The base directory of a NiFi / NiFi Registry installation that should be encrypted
+     */
+    public int encryptConfigurationFiles(final Path baseDirectory, final PropertyProtectionScheme scheme) {

Review Comment:
   Returning an `int` from this method does not provide a very clear indication of the status. Recommend at least using an enum, and incorporating a numeric status if needed.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/crypto/CryptographyUtils.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.util.crypto;
+
+import org.apache.commons.codec.binary.Hex;
+import org.bouncycastle.crypto.generators.SCrypt;
+
+import javax.crypto.Cipher;
+import java.nio.charset.StandardCharsets;
+import java.security.KeyException;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.List;
+
+public class CryptographyUtils {
+
+    private static final String NIFI_SCRYPT_SALT = "NIFI_SCRYPT_SALT";
+    private static final int DEFAULT_MIN_PASSPHRASE_LENGTH = 12;
+
+    // Strong parameters as of 12 Aug 2016
+    private static final int SCRYPT_N = (int) Math.pow(2, 16);
+    private static final int SCRYPT_R = 8;
+    private static final int SCRYPT_P = 1;
+
+    public static String deriveKeyFromPassphrase(final String passphrase) throws KeyException, NoSuchAlgorithmException {
+        return deriveKeyFromPassphrase(passphrase, DEFAULT_MIN_PASSPHRASE_LENGTH);
+    }
+
+    public static boolean isUnlimitedStrengthCryptoAvailable() {
+        try {
+            return Cipher.getMaxAllowedKeyLength("AES") > 128;
+        } catch (NoSuchAlgorithmException e) {
+            return false;
+        }
+    }
+
+    private static String deriveKeyFromPassphrase(final String passphrase, final int minPassphraseLength) throws KeyException, NoSuchAlgorithmException {
+        final String trimmedPassphrase = passphrase.trim();
+        if (trimmedPassphrase.length() < minPassphraseLength) {
+            throw new KeyException(String.format("Cannot derive key from empty/short passphrase -- passphrase must be at least %d characters", DEFAULT_MIN_PASSPHRASE_LENGTH));
+        }
+
+        // Generate a 128 bit salt
+        byte[] salt = generateScryptSalt();
+        int keyLengthInBytes = getValidKeyLengths().stream().max(Integer::compare).get() / 8;
+        byte[] derivedKeyBytes = SCrypt.generate(passphrase.getBytes(StandardCharsets.UTF_8), salt, SCRYPT_N, SCRYPT_R, SCRYPT_P, keyLengthInBytes);
+        return Hex.encodeHexString(derivedKeyBytes).toUpperCase();

Review Comment:
   This seems like an opportunity to introduce support for a different algorithm. It is probably unnecessary to support the same level of configuration as currently supported for the Sensitive Properties Algorithm, but perhaps standardizing on PBKDF2 would be better, as long as there is a way to maintain compatibility with current implementations.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/file/ConfigurationFileResolver.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.util.file;
+
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.util.NiFiProperties;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Resolve configuration files that need to be encrypted from a given ApplicationProperties
+ */
+public class ConfigurationFileResolver {

Review Comment:
   As noted with other classes, this looks like a good opportunity to define an interface and then provide a standard implementation.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/PropertyEncryptorMain.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * 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;
+
+import org.apache.nifi.properties.AbstractBootstrapPropertiesLoader;
+import org.apache.nifi.properties.BootstrapProperties;
+import org.apache.nifi.properties.MutableBootstrapProperties;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.StandardSensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.registry.properties.util.NiFiRegistryBootstrapPropertiesLoader;
+import org.apache.nifi.serde.PropertiesWriter;
+import org.apache.nifi.util.NiFiBootstrapPropertiesLoader;
+import org.apache.nifi.util.crypto.CryptographyUtils;
+import org.apache.nifi.xml.XmlDecryptor;
+import org.apache.nifi.xml.XmlEncryptor;
+import org.apache.nifi.util.file.ConfigurationFileResolver;
+import org.apache.nifi.util.file.FileUtilities;
+import org.apache.nifi.properties.ApplicationProperties;
+import org.apache.nifi.properties.NiFiPropertiesLoader;
+import org.apache.nifi.properties.PropertiesLoader;
+import org.apache.nifi.registry.properties.NiFiRegistryPropertiesLoader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.List;
+
+public class PropertyEncryptorMain {
+
+    private static final Logger logger = LoggerFactory.getLogger(PropertyEncryptorMain.class);
+    private final AbstractBootstrapPropertiesLoader bootstrapLoader;
+    private final PropertiesLoader<ApplicationProperties> propertiesLoader;
+    private final ConfigurationFileResolver fileResolver;
+    private final List<File> configurationFiles;
+    private String hexKey;
+    private final Path confDirectory;
+
+    public PropertyEncryptorMain(final Path baseDirectory, final String passphrase) throws PropertyEncryptorException {
+        confDirectory = FileUtilities.resolveAbsoluteConfDirectory(baseDirectory);
+        try {
+            bootstrapLoader = getBootstrapPropertiesLoader(confDirectory);
+            fileResolver = new ConfigurationFileResolver(confDirectory);
+            final File applicationProperties = FileUtilities.resolvePropertiesFile(confDirectory);
+            propertiesLoader = getPropertiesLoader(confDirectory);
+            configurationFiles = fileResolver.resolveConfigurationFilesFromApplicationProperties(propertiesLoader.load(applicationProperties));
+            hexKey = getKeyHex(confDirectory, passphrase);
+        } catch (final Exception e) {
+            throw new PropertyEncryptorException("Failed to run property encryptor", e);
+        }
+    }
+
+    private AbstractBootstrapPropertiesLoader getBootstrapPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryBootstrapPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiBootstrapPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized bootstrap.conf file");
+        }
+    }
+
+    private PropertiesLoader<ApplicationProperties> getPropertiesLoader(final Path baseDirectory) {
+        if (FileUtilities.isNiFiRegistryConfDirectory(baseDirectory)) {
+            return new NiFiRegistryPropertiesLoader();
+        } else if (FileUtilities.isNiFiConfDirectory(baseDirectory)) {
+            return new NiFiPropertiesLoader();
+        } else {
+            throw new PropertyEncryptorException("The base directory provided does not contain a recognized .properties file");
+        }
+    }
+
+    /**
+     * @param baseDirectory The base directory of a NiFi / NiFi Registry installation that should be encrypted
+     */
+    public int encryptConfigurationFiles(final Path baseDirectory, final PropertyProtectionScheme scheme) {
+        XmlEncryptor encryptor = getXmlEncryptor(scheme);
+        try {
+            encryptConfigurationFiles(configurationFiles, encryptor);
+            outputKeyToBootstrap();
+            logger.info("The Property Encryptor successfully encrypted configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath());
+            return 0;
+        } catch (Exception e) {
+            logger.error("The Property Encryptor failed to encrypt configuration files in the [{}] directory with the scheme [{}]", baseDirectory, scheme.getPath(), e);
+            return 1;
+        }
+    }
+
+    private void outputKeyToBootstrap() throws IOException {
+        final File bootstrapFile = bootstrapLoader.getBootstrapFileWithinConfDirectory(confDirectory);
+        File tempBootstrapFile = FileUtilities.getTemporaryOutputFile("tmp", bootstrapFile);
+        final MutableBootstrapProperties bootstrapProperties = bootstrapLoader.loadMutableBootstrapProperties(bootstrapFile.getPath());
+        bootstrapProperties.setProperty(BootstrapProperties.BootstrapPropertyKey.SENSITIVE_KEY.getKey(), hexKey);
+        PropertiesWriter.writePropertiesFile(new FileInputStream(bootstrapFile), new FileOutputStream(tempBootstrapFile), bootstrapProperties);

Review Comment:
   It looks like these streams should be created in a try-with-resources block.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoException.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.xml;
+
+public class XmlCryptoException extends RuntimeException {

Review Comment:
   This seems a bit more specific than necessary, perhaps `CryptoException`?



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/file/FileUtilities.java:
##########
@@ -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.util.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+
+public class FileUtilities {

Review Comment:
   This class name seems a bit too generic. Perhaps ConfigurationFileUtils?



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {
+
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+    protected static final String PARENT_IDENTIFIER = "identifier";
+    protected static final String PROPERTY_ELEMENT = "property";
+
+    protected final SensitivePropertyProvider cryptoProvider;
+    protected SensitivePropertyProviderFactory providerFactory;
+
+    public XmlCryptoParser(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        this.providerFactory = providerFactory;
+        cryptoProvider = providerFactory.getProvider(scheme);
+    }
+
+    protected void cryptographicXmlOperation(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        final XMLOutputFactory factory = XMLOutputFactory.newInstance();
+        factory.setProperty("com.ctc.wstx.outputValidateStructure", false);
+
+        try {
+            final XMLEventReader eventReader = getXMLReader(encryptedXmlContent);
+            final XMLEventWriter xmlWriter = factory.createXMLEventWriter(decryptedOutputStream);
+            String groupIdentifier = "";
+
+            while(eventReader.hasNext()) {
+                XMLEvent event = eventReader.nextEvent();
+
+                if (isGroupIdentifier(event)) {
+                    groupIdentifier = getGroupIdentifier(eventReader.nextEvent());
+                }
+
+                if (isSensitiveElement(event)) {
+                    xmlWriter.add(updateStartElementEncryptionAttribute(event));
+                    xmlWriter.add(cryptoOperationOnCharacters(eventReader.nextEvent(), groupIdentifier, getPropertyName(event)));
+                } else {
+                    try {
+                        xmlWriter.add(event);
+                    } catch (Exception e) {
+                        throw new RuntimeException("Failed operation on XML content", e);

Review Comment:
   It would be helpful to throw a more specific type of exception.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {
+
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+    protected static final String PARENT_IDENTIFIER = "identifier";
+    protected static final String PROPERTY_ELEMENT = "property";
+
+    protected final SensitivePropertyProvider cryptoProvider;
+    protected SensitivePropertyProviderFactory providerFactory;
+
+    public XmlCryptoParser(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        this.providerFactory = providerFactory;
+        cryptoProvider = providerFactory.getProvider(scheme);
+    }
+
+    protected void cryptographicXmlOperation(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        final XMLOutputFactory factory = XMLOutputFactory.newInstance();
+        factory.setProperty("com.ctc.wstx.outputValidateStructure", false);

Review Comment:
   This property presumes the use of Woodstox, which is not clearly implied elsewhere. It looks like this should be removed and re-evaluated.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {
+
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+    protected static final String PARENT_IDENTIFIER = "identifier";
+    protected static final String PROPERTY_ELEMENT = "property";
+
+    protected final SensitivePropertyProvider cryptoProvider;
+    protected SensitivePropertyProviderFactory providerFactory;
+
+    public XmlCryptoParser(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        this.providerFactory = providerFactory;
+        cryptoProvider = providerFactory.getProvider(scheme);
+    }
+
+    protected void cryptographicXmlOperation(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        final XMLOutputFactory factory = XMLOutputFactory.newInstance();
+        factory.setProperty("com.ctc.wstx.outputValidateStructure", false);
+
+        try {
+            final XMLEventReader eventReader = getXMLReader(encryptedXmlContent);
+            final XMLEventWriter xmlWriter = factory.createXMLEventWriter(decryptedOutputStream);
+            String groupIdentifier = "";
+
+            while(eventReader.hasNext()) {
+                XMLEvent event = eventReader.nextEvent();
+
+                if (isGroupIdentifier(event)) {
+                    groupIdentifier = getGroupIdentifier(eventReader.nextEvent());
+                }
+
+                if (isSensitiveElement(event)) {
+                    xmlWriter.add(updateStartElementEncryptionAttribute(event));
+                    xmlWriter.add(cryptoOperationOnCharacters(eventReader.nextEvent(), groupIdentifier, getPropertyName(event)));
+                } else {
+                    try {
+                        xmlWriter.add(event);
+                    } catch (Exception e) {
+                        throw new RuntimeException("Failed operation on XML content", e);
+                    }
+                }
+            }
+
+            eventReader.close();
+            xmlWriter.flush();
+            xmlWriter.close();
+        } catch (Exception e) {
+            throw new RuntimeException("Failed operation on XML content", e);
+        }
+    }
+
+    /**
+     * Update the StartElement 'encryption' attribute for a sensitive value to add or remove the respective encryption details eg. encryption="aes/gcm/128"
+     * @param xmlEvent A 'sensitive' StartElement that contains the 'encryption' tag attribute
+     * @return The updated StartElement
+     */
+    protected abstract StartElement updateStartElementEncryptionAttribute(final XMLEvent xmlEvent);
+
+    /**
+     * Perform an encrypt or decrypt cryptographic operation on a Characters element
+     * @param xmlEvent A Characters XmlEvent
+     * @param groupIdentifier The XML <identifier/> tag
+     * @return The Characters XmlEvent that has been updated by the cryptographic operation
+     */
+    protected abstract Characters cryptoOperationOnCharacters(final XMLEvent xmlEvent, final String groupIdentifier, final String propertyName);
+
+    private String getGroupIdentifier(final XMLEvent xmlEvent) {
+        if (xmlEvent.isCharacters()) {
+            return xmlEvent.asCharacters().getData();
+        } else {
+            return "";
+        }
+    }
+
+    protected String getPropertyName(final XMLEvent xmlEvent) {
+        return xmlEvent.asStartElement().getName().toString();
+    }
+
+    protected boolean isGroupIdentifier(final XMLEvent xmlEvent) {
+        return xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PARENT_IDENTIFIER);
+    }
+
+    protected boolean isSensitiveElement(final XMLEvent xmlEvent) {
+        return  xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PROPERTY_ELEMENT)
+                && elementHasEncryptionAttribute(xmlEvent.asStartElement());
+    }
+
+    protected XMLEventReader getXMLReader(final InputStream fileStream) throws XMLStreamException, FileNotFoundException {
+        XMLInputFactory xmlInputFactory = XMLInputFactory.newFactory();
+        xmlInputFactory.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+        xmlInputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+
+        return xmlInputFactory.createXMLEventReader(fileStream);
+    }

Review Comment:
   This should be replaced with usage of XML components from `nifi-xml-processing` to avoid the possibility of external entity parsing.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/test/resources/login-identity-providers-populated-encrypted.xml:
##########
@@ -0,0 +1,102 @@
+<?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 login identity providers to use when running securely. In order
+    to use a specific provider it must be configured here and it's identifier
+    must be specified in the nifi.properties file.
+--><loginIdentityProviders>
+    <!--
+        Identity Provider for users logging in with username/password against an LDAP server.
+
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, 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 START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using 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' - Url of the LDAP servier (i.e. ldap://<hostname>:<port>).
+        'User Search Base' - Base DN for searching for users (i.e. CN=Users,DC=example,DC=com).
+        'User Search Filter' - Filter for searching for users against the 'User Search Base'.
+            (i.e. sAMAccountName={0}). The user specified name is inserted into '{0}'.
+
+        'Authentication Expiration' - The duration of how long the user authentication is valid
+            for. If the user never logs out, they will be required to log back in following
+            this duration.
+    -->

Review Comment:
   The file contents should be reduced to the minimum, so comments like this can be removed.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {
+
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+    protected static final String PARENT_IDENTIFIER = "identifier";
+    protected static final String PROPERTY_ELEMENT = "property";
+
+    protected final SensitivePropertyProvider cryptoProvider;
+    protected SensitivePropertyProviderFactory providerFactory;
+
+    public XmlCryptoParser(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        this.providerFactory = providerFactory;
+        cryptoProvider = providerFactory.getProvider(scheme);
+    }
+
+    protected void cryptographicXmlOperation(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        final XMLOutputFactory factory = XMLOutputFactory.newInstance();
+        factory.setProperty("com.ctc.wstx.outputValidateStructure", false);
+
+        try {
+            final XMLEventReader eventReader = getXMLReader(encryptedXmlContent);
+            final XMLEventWriter xmlWriter = factory.createXMLEventWriter(decryptedOutputStream);
+            String groupIdentifier = "";
+
+            while(eventReader.hasNext()) {
+                XMLEvent event = eventReader.nextEvent();
+
+                if (isGroupIdentifier(event)) {
+                    groupIdentifier = getGroupIdentifier(eventReader.nextEvent());
+                }
+
+                if (isSensitiveElement(event)) {
+                    xmlWriter.add(updateStartElementEncryptionAttribute(event));
+                    xmlWriter.add(cryptoOperationOnCharacters(eventReader.nextEvent(), groupIdentifier, getPropertyName(event)));
+                } else {
+                    try {
+                        xmlWriter.add(event);
+                    } catch (Exception e) {
+                        throw new RuntimeException("Failed operation on XML content", e);
+                    }
+                }
+            }
+
+            eventReader.close();
+            xmlWriter.flush();
+            xmlWriter.close();
+        } catch (Exception e) {
+            throw new RuntimeException("Failed operation on XML content", e);
+        }
+    }
+
+    /**
+     * Update the StartElement 'encryption' attribute for a sensitive value to add or remove the respective encryption details eg. encryption="aes/gcm/128"
+     * @param xmlEvent A 'sensitive' StartElement that contains the 'encryption' tag attribute
+     * @return The updated StartElement
+     */
+    protected abstract StartElement updateStartElementEncryptionAttribute(final XMLEvent xmlEvent);
+
+    /**
+     * Perform an encrypt or decrypt cryptographic operation on a Characters element
+     * @param xmlEvent A Characters XmlEvent
+     * @param groupIdentifier The XML <identifier/> tag
+     * @return The Characters XmlEvent that has been updated by the cryptographic operation
+     */
+    protected abstract Characters cryptoOperationOnCharacters(final XMLEvent xmlEvent, final String groupIdentifier, final String propertyName);
+
+    private String getGroupIdentifier(final XMLEvent xmlEvent) {
+        if (xmlEvent.isCharacters()) {
+            return xmlEvent.asCharacters().getData();
+        } else {
+            return "";
+        }
+    }
+
+    protected String getPropertyName(final XMLEvent xmlEvent) {
+        return xmlEvent.asStartElement().getName().toString();
+    }
+
+    protected boolean isGroupIdentifier(final XMLEvent xmlEvent) {
+        return xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PARENT_IDENTIFIER);
+    }
+
+    protected boolean isSensitiveElement(final XMLEvent xmlEvent) {
+        return  xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PROPERTY_ELEMENT)
+                && elementHasEncryptionAttribute(xmlEvent.asStartElement());
+    }
+
+    protected XMLEventReader getXMLReader(final InputStream fileStream) throws XMLStreamException, FileNotFoundException {
+        XMLInputFactory xmlInputFactory = XMLInputFactory.newFactory();
+        xmlInputFactory.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+        xmlInputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+
+        return xmlInputFactory.createXMLEventReader(fileStream);
+    }
+
+    private boolean elementHasEncryptionAttribute(final StartElement xmlEvent) {
+        return xmlElementHasAttribute(xmlEvent, ENCRYPTION_ATTRIBUTE_NAME);
+    }
+
+    private boolean xmlElementHasAttribute(final StartElement xmlEvent, final String attributeName) {
+        return !Objects.isNull(xmlEvent.getAttributeByName(new QName(attributeName)));
+    }
+
+    protected StartElement updateElementAttribute(final XMLEvent xmlEvent, final String attributeName, final String attributeValue) {

Review Comment:
   The general order of methods should be public, protected, then private.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/test/resources/login-identity-providers-populated-unecrypted.xml:
##########
@@ -0,0 +1,102 @@
+<?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 login identity providers to use when running securely. In order
+    to use a specific provider it must be configured here and it's identifier
+    must be specified in the nifi.properties file.
+--><loginIdentityProviders>
+    <!--
+        Identity Provider for users logging in with username/password against an LDAP server.
+
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, 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 START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using 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' - Url of the LDAP servier (i.e. ldap://<hostname>:<port>).
+        'User Search Base' - Base DN for searching for users (i.e. CN=Users,DC=example,DC=com).
+        'User Search Filter' - Filter for searching for users against the 'User Search Base'.
+            (i.e. sAMAccountName={0}). The user specified name is inserted into '{0}'.
+
+        'Authentication Expiration' - The duration of how long the user authentication is valid
+            for. If the user never logs out, they will be required to log back in following
+            this duration.
+    -->

Review Comment:
   ```suggestion
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r960075378


##########
nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/PropertyProtectionScheme.java:
##########
@@ -19,7 +19,7 @@
 /**
  * Property Protection Schemes supported as arguments for encryption commands should not have direct references
  */
-enum PropertyProtectionScheme implements ProtectionScheme {
+public enum PropertyProtectionScheme implements ProtectionScheme {

Review Comment:
   I added a SchemeCandidates class which utilises the StandardProtectionSchemeResolver instead of directly using the PropertyProtectionScheme enum. Does this resolve the issue at hand? There may still be future work to allow providing any extra required parameters to handle these other schemes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r959786011


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/ConfigSubcommand.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.util.console;
+
+import org.apache.nifi.PropertyEncryptorMain;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.util.console.utils.BaseCommandParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+@CommandLine.Command(name = "config",
+        description = "Operate on application configs",
+        usageHelpWidth=140
+)
+class ConfigSubcommand extends BaseCommandParameters implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConfigSubcommand.class);
+
+    @CommandLine.ParentCommand
+    private DefaultCLIOptions parent;
+
+    @CommandLine.Parameters(description="The encryption scheme to use, from one of the following schemes: [@|bold ${COMPLETION-CANDIDATES}|@]")
+    PropertyProtectionScheme scheme;

Review Comment:
   I'm wondering if for this PR we might only support AES and worry about other schemes later?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r958730858


##########
nifi-toolkit/nifi-property-encryptor-tool/pom.xml:
##########
@@ -0,0 +1,135 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-toolkit</artifactId>
+        <version>1.18.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>nifi-property-encryptor-tool</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>info.picocli</groupId>
+            <artifactId>picocli</artifactId>
+            <version>4.6.3</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-property-protection-factory</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-properties-loader</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-properties-loader</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>${org.slf4j.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <version>${org.slf4j.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-xml-processing</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <scope>compile</scope>

Review Comment:
   The `compile` scope can be removed in general, since it the default setting.



##########
nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/PropertyProtectionScheme.java:
##########
@@ -19,7 +19,7 @@
 /**
  * Property Protection Schemes supported as arguments for encryption commands should not have direct references
  */
-enum PropertyProtectionScheme implements ProtectionScheme {
+public enum PropertyProtectionScheme implements ProtectionScheme {

Review Comment:
   Recommend revisiting the implementation approach, this enum should remain package-private and references to a `ProtectionScheme` should be accessed using the associated Resolver.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/ConfigSubcommand.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.util.console;
+
+import org.apache.nifi.PropertyEncryptorCommand;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.util.console.utils.BaseCommandParameters;
+import org.apache.nifi.util.console.utils.SchemeCandidates;
+import org.apache.nifi.util.console.utils.SchemeConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+
+@CommandLine.Command(name = "config",
+        description = "Operate on application configs",
+        usageHelpWidth=140
+)
+class ConfigSubcommand extends BaseCommandParameters implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConfigSubcommand.class);
+    private static final String runMessage = "The property encryptor is running to [{}] configuration files in [{}]";
+
+    @CommandLine.ParentCommand
+    private BaseCLICommand parent;
+
+    @CommandLine.Parameters(
+            completionCandidates = SchemeCandidates.class,
+            converter = SchemeConverter.class,
+            description="The encryption scheme to use, from one of the following schemes: [@|bold ${COMPLETION-CANDIDATES}|@]")
+    ProtectionScheme scheme;
+
+    @Override
+    public void run() {
+        final PropertyEncryptorCommand propertyEncryptorCommand = new PropertyEncryptorCommand(baseDirectory, passphrase);
+        if (parent instanceof PropertyEncryptorEncrypt) {
+            encryptConfiguration(propertyEncryptorCommand);
+        } else if (parent instanceof PropertyEncryptorDecrypt) {
+            logger.info(runMessage, "decrypt", baseDirectory);
+        } else if (parent instanceof PropertyEncryptorMigrate) {
+            logger.info(runMessage, "migrate", baseDirectory);
+        }
+    }
+
+    private void encryptConfiguration(final PropertyEncryptorCommand propertyEncryptorCommand) {
+        logger.info(runMessage, "encrypt", baseDirectory);
+        propertyEncryptorCommand.encryptXmlConfigurationFiles(baseDirectory, scheme);
+        try {
+            propertyEncryptorCommand.encryptPropertiesFile(scheme);
+            propertyEncryptorCommand.outputKeyToBootstrap();
+        } catch (IOException e) {
+            e.printStackTrace();

Review Comment:
   It looks like this should be replaced with `logger.error()`



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/serde/StandardPropertiesWriter.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.serde;
+
+import org.apache.nifi.properties.ReadableProperties;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Set;
+
+public class StandardPropertiesWriter implements PropertiesWriter {
+
+    private static final String DELIMITER = "=";
+    private static final String PROPERTY_FORMAT = "%s=%s";
+
+    public void writePropertiesFile(final InputStream inputStream, final OutputStream outputStream, final ReadableProperties properties) throws IOException {
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream));
+             BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(outputStream))) {
+            String line;
+            while ((line = reader.readLine()) != null) {
+                Set<String> keys = properties.getPropertyKeys();
+                for (final String key : keys) {
+                    if (line.split(DELIMITER)[0].matches(key)) {

Review Comment:
   Instead of calling `line.split()` and then `matches`, this could be adjusted to compile a run a regular expression pattern using the `key`, which seems like it would be a bit more straightforward.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/ConfigSubcommand.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.util.console;
+
+import org.apache.nifi.PropertyEncryptorCommand;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+import org.apache.nifi.util.console.utils.BaseCommandParameters;
+import org.apache.nifi.util.console.utils.SchemeCandidates;
+import org.apache.nifi.util.console.utils.SchemeConverter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+import java.io.IOException;
+
+@CommandLine.Command(name = "config",
+        description = "Operate on application configs",
+        usageHelpWidth=140
+)
+class ConfigSubcommand extends BaseCommandParameters implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConfigSubcommand.class);
+    private static final String runMessage = "The property encryptor is running to [{}] configuration files in [{}]";

Review Comment:
   This should be uppercased to follow standard conventions.
   ```suggestion
       private static final String RUN_LOG_MESSAGE = "The property encryptor is running to [{}] configuration files in [{}]";
   ```



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/file/ConfigurationFileUtils.java:
##########
@@ -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.util.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+
+public class ConfigurationFileUtils {
+
+    public static String DEFAULT_CONF_DIR = "conf";
+    public static String NIFI_PROPERTIES_DEFAULT_NAME = "nifi.properties";
+    public static String NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME = "nifi-registry.properties";
+
+    public static File getTemporaryOutputFile(final String prefix, final File siblingFile) throws IOException {
+        if (siblingFile != null && siblingFile.isFile()) {
+            return File.createTempFile(prefix, siblingFile.getName(), siblingFile.getParentFile());

Review Comment:
   The documentation for `java.io.File.createTempFile()` recommends using `java.nio.Files.createTempFile()` because it has more restrictive default permissions, so recommend changing the implementation to use `java.nio.Files.createTempFile()`.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/utils/BaseCommandParameters.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.util.console.utils;
+
+import picocli.CommandLine;
+
+import java.nio.file.Path;
+
+public class BaseCommandParameters {
+    @CommandLine.Parameters(description="The base directory of NiFi/NiFi Registry/MiNiFi which contains the 'conf' directory (eg. /var/lib/nifi)")
+    protected Path baseDirectory;
+
+    @CommandLine.Parameters(description="The passphrase used to derive a key and encrypt files (12 characters minimum)")
+    protected String passphrase;

Review Comment:
   The parameter and description do not make the purpose clear. Is this for the Bootstrap Root Key? If so, recommend naming it `rootPassphrase` and updating the description.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/file/ConfigurationFileUtils.java:
##########
@@ -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.util.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+
+public class ConfigurationFileUtils {
+
+    public static String DEFAULT_CONF_DIR = "conf";
+    public static String NIFI_PROPERTIES_DEFAULT_NAME = "nifi.properties";
+    public static String NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME = "nifi-registry.properties";
+
+    public static File getTemporaryOutputFile(final String prefix, final File siblingFile) throws IOException {
+        if (siblingFile != null && siblingFile.isFile()) {
+            return File.createTempFile(prefix, siblingFile.getName(), siblingFile.getParentFile());
+        } else {
+            throw new IOException("Failed to create temporary output file because sibling file is null or is not a file");
+        }
+    }
+
+    public static boolean isSafeToWrite(final File fileToWrite) {
+        assert(fileToWrite != null);
+        return (!fileToWrite.exists() && fileToWrite.getParentFile().canWrite() || (fileToWrite.exists() && fileToWrite.canWrite()));
+    }
+
+    public static boolean isNiFiConfDirectory(final Path baseDirectory) {
+        return directoryContainsFilename(baseDirectory, NIFI_PROPERTIES_DEFAULT_NAME);
+    }
+
+    public static boolean isNiFiRegistryConfDirectory(final Path baseDirectory) {
+        return directoryContainsFilename(baseDirectory, NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME);
+    }
+
+    /**
+     * If the baseDirectory given is ./nifi/conf, return the parent directory ./nifi
+     * @param baseDirectory A given base directory to locate configuration files
+     * @return The ./nifi directory
+     */
+    public static Path resolveAbsoluteConfDirectory(final Path baseDirectory) {
+        if (!baseDirectory.toFile().isDirectory()) {
+            throw new IllegalArgumentException(String.format("The base directory given [%s] does not exist or is not a directory", baseDirectory));
+        }
+
+        if (isNiFiConfDirectory(baseDirectory) || isNiFiRegistryConfDirectory(baseDirectory)) {
+            return getAbsolutePath(baseDirectory);
+        } else if (directoryContainsFilename(baseDirectory, DEFAULT_CONF_DIR)) {
+            return getAbsolutePath(getDefaultConfDirectory(baseDirectory).toPath());
+        } else {
+            throw new IllegalArgumentException(
+                    String.format("The configuration directory [%s]/ could not be found within [%s] or it did not contain a properties file", DEFAULT_CONF_DIR, baseDirectory));

Review Comment:
   Should the forward slash be removed?
   ```suggestion
                       String.format("The configuration directory [%s] could not be found within [%s] or it did not contain a properties file", DEFAULT_CONF_DIR, baseDirectory));
   ```



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/file/ConfigurationFileUtils.java:
##########
@@ -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.util.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+
+public class ConfigurationFileUtils {
+
+    public static String DEFAULT_CONF_DIR = "conf";
+    public static String NIFI_PROPERTIES_DEFAULT_NAME = "nifi.properties";
+    public static String NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME = "nifi-registry.properties";
+
+    public static File getTemporaryOutputFile(final String prefix, final File siblingFile) throws IOException {
+        if (siblingFile != null && siblingFile.isFile()) {
+            return File.createTempFile(prefix, siblingFile.getName(), siblingFile.getParentFile());
+        } else {
+            throw new IOException("Failed to create temporary output file because sibling file is null or is not a file");

Review Comment:
   It would be helpful to include the prefix and sibling file parameters in the message.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/file/ConfigurationFileUtils.java:
##########
@@ -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.util.file;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+
+public class ConfigurationFileUtils {
+
+    public static String DEFAULT_CONF_DIR = "conf";
+    public static String NIFI_PROPERTIES_DEFAULT_NAME = "nifi.properties";
+    public static String NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME = "nifi-registry.properties";
+
+    public static File getTemporaryOutputFile(final String prefix, final File siblingFile) throws IOException {
+        if (siblingFile != null && siblingFile.isFile()) {
+            return File.createTempFile(prefix, siblingFile.getName(), siblingFile.getParentFile());
+        } else {
+            throw new IOException("Failed to create temporary output file because sibling file is null or is not a file");
+        }
+    }
+
+    public static boolean isSafeToWrite(final File fileToWrite) {
+        assert(fileToWrite != null);
+        return (!fileToWrite.exists() && fileToWrite.getParentFile().canWrite() || (fileToWrite.exists() && fileToWrite.canWrite()));
+    }
+
+    public static boolean isNiFiConfDirectory(final Path baseDirectory) {
+        return directoryContainsFilename(baseDirectory, NIFI_PROPERTIES_DEFAULT_NAME);
+    }
+
+    public static boolean isNiFiRegistryConfDirectory(final Path baseDirectory) {
+        return directoryContainsFilename(baseDirectory, NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME);
+    }
+
+    /**
+     * If the baseDirectory given is ./nifi/conf, return the parent directory ./nifi
+     * @param baseDirectory A given base directory to locate configuration files
+     * @return The ./nifi directory
+     */
+    public static Path resolveAbsoluteConfDirectory(final Path baseDirectory) {
+        if (!baseDirectory.toFile().isDirectory()) {
+            throw new IllegalArgumentException(String.format("The base directory given [%s] does not exist or is not a directory", baseDirectory));
+        }
+
+        if (isNiFiConfDirectory(baseDirectory) || isNiFiRegistryConfDirectory(baseDirectory)) {
+            return getAbsolutePath(baseDirectory);
+        } else if (directoryContainsFilename(baseDirectory, DEFAULT_CONF_DIR)) {
+            return getAbsolutePath(getDefaultConfDirectory(baseDirectory).toPath());
+        } else {
+            throw new IllegalArgumentException(
+                    String.format("The configuration directory [%s]/ could not be found within [%s] or it did not contain a properties file", DEFAULT_CONF_DIR, baseDirectory));
+        }
+    }
+
+    /**
+     * Get the properties file either NiFi or NiFi Registry from within the configuration directory
+     * @param confDirectory The ./conf directory
+     * @return The NiFi or NiFi Registry properties file (eg. nifi.properties or nifi-registry.properties)
+     */
+    public static File resolvePropertiesFile(final Path confDirectory) {
+        if (directoryContainsFilename(confDirectory, NIFI_PROPERTIES_DEFAULT_NAME)) {
+            return getAbsolutePath(confDirectory.resolve(NIFI_PROPERTIES_DEFAULT_NAME)).toFile();
+        } else if (directoryContainsFilename(confDirectory, NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME)) {
+            return getAbsolutePath(confDirectory.resolve(NIFI_REGISTRY_DEFAULT_PROPERTIES_NAME)).toFile();
+        } else {
+            throw new IllegalArgumentException(String.format("Could not find a properties file in [%s]", confDirectory));
+        }
+    }
+
+    private static Path getAbsolutePath(final Path relativeFile) {
+        final Path absolutePath = relativeFile.toAbsolutePath();
+        if (absolutePath.toFile().exists() && absolutePath.toFile().canRead()) {
+            return absolutePath;
+        } else {
+            throw new IllegalArgumentException(String.format("The file or directory [%s] does not exist", absolutePath));
+        }
+    }
+
+    private static boolean directoryContainsFilename(final Path directory, final String filename) {
+        return Arrays.stream(directory.toFile().listFiles()).anyMatch(file -> file.getName().equals(filename));
+    }
+
+    private static File getDefaultConfDirectory(final Path baseDirectory) {
+        return baseDirectory.resolve(DEFAULT_CONF_DIR).toFile();
+    }
+
+    /**
+     * Return a configuration file absolute path based on the confDirectory rather than Java's working path
+     */
+    public static File getAbsoluteFile(final File confDirectory, final File relativeFile) {

Review Comment:
   This method should be moved up in the file before other `private` methods.



##########
nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/PropertyProtectionScheme.java:
##########
@@ -19,7 +19,7 @@
 /**
  * Property Protection Schemes supported as arguments for encryption commands should not have direct references
  */
-enum PropertyProtectionScheme implements ProtectionScheme {
+public enum PropertyProtectionScheme implements ProtectionScheme {

Review Comment:
   Thanks for the update @thenatog, that looks like a good solution. I will take a closer look at the other changes soon.



##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/file/NiFiRegistryConfigurationFileResolver.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.util.file;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Resolve configuration files that need to be encrypted from a given ApplicationProperties
+ */
+public class NiFiRegistryConfigurationFileResolver implements ConfigurationFileResolver<NiFiRegistryProperties> {
+
+    private Path confDirectory;
+
+    public NiFiRegistryConfigurationFileResolver(final Path confDirectory) {
+        this.confDirectory = confDirectory;
+    }
+
+    /**
+     * Use the nifi.properties file to locate configuration files referenced by properties in the file
+     *
+     * @return List of application configuration files
+     */
+    @Override
+    public List<File> resolveConfigurationFilesFromApplicationProperties(NiFiRegistryProperties properties) throws ConfigurationFileResolverException {
+        ArrayList<File> configurationFiles = new ArrayList<>();
+        configurationFiles.add(ConfigurationFileUtils.getAbsoluteFile(confDirectory.toFile(), properties.getAuthorizersConfigurationFile()));
+        configurationFiles.add(ConfigurationFileUtils.getAbsoluteFile(confDirectory.toFile(), properties.getProvidersConfigurationFile()));
+        configurationFiles.add(ConfigurationFileUtils.getAbsoluteFile(confDirectory.toFile(), properties.getIdentityProviderConfigurationFile()));
+        configurationFiles.add(ConfigurationFileUtils.getAbsoluteFile(confDirectory.toFile(), properties.getRegistryAliasConfigurationFile()));
+
+        for (final File configFile : configurationFiles) {
+            if (!isValidConfigurationFile(configFile)) {
+                throw new ConfigurationFileResolverException(String.format("Failed to resolve configuration file [%s].", configFile.getName()));

Review Comment:
   Recommend moving the period character from the message:
   ```suggestion
                   throw new ConfigurationFileResolverException(String.format("Failed to resolve configuration file [%s]", configFile.getName()));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r1033694205


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/utils/SchemeCandidates.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.util.console.utils;
+
+import org.apache.nifi.properties.scheme.StandardProtectionSchemeResolver;
+
+import java.util.ArrayList;
+
+public class SchemeCandidates extends ArrayList<String> {
+    SchemeCandidates() {
+        super(new StandardProtectionSchemeResolver().getSupportedProtectionSchemes());

Review Comment:
   Is there an existing way to enumerate all the possible ProtectionScheme paths available at run time? This SchemeCandidates class is used to present all available options to the user in the CLI and does validation checking on the user selection of the scheme



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r955331683


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {

Review Comment:
   Reusing the code makes sense, but the general contract is not clear. The concrete classes have `decrypt` and `encrypt` public methods, so perhaps those methods should be called out in interface definitions. On the other hand, perhaps a more generic method name would work, if both methods have similar inputs and outputs. On further review, having these shared protected methods is not necessarily a problem.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r967495476


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/serde/StandardPropertiesWriter.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.serde;
+
+import org.apache.nifi.properties.ReadableProperties;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.Set;
+
+public class StandardPropertiesWriter implements PropertiesWriter {
+
+    private static final String DELIMITER = "=";
+    private static final String PROPERTY_FORMAT = "%s=%s";
+
+    public void writePropertiesFile(final InputStream inputStream, final OutputStream outputStream, final ReadableProperties properties) throws IOException {
+        try (BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream));
+             BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(outputStream))) {
+            String line;
+            while ((line = reader.readLine()) != null) {
+                Set<String> keys = properties.getPropertyKeys();
+                for (final String key : keys) {
+                    if (line.split(DELIMITER)[0].matches(key)) {

Review Comment:
   Updated this, not sure if I did this quite the way you described, let me know if it should change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on PR #6273:
URL: https://github.com/apache/nifi/pull/6273#issuecomment-1489276883

   With the shift towards NiFi 2.0 and planned consolidation of Flow Configuration files, plus deprecation and removal of legacy algorithms, the implementation will need to be revisited.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r953140358


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {
+
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+    protected static final String PARENT_IDENTIFIER = "identifier";
+    protected static final String PROPERTY_ELEMENT = "property";
+
+    protected final SensitivePropertyProvider cryptoProvider;
+    protected SensitivePropertyProviderFactory providerFactory;
+
+    public XmlCryptoParser(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        this.providerFactory = providerFactory;
+        cryptoProvider = providerFactory.getProvider(scheme);
+    }
+
+    protected void cryptographicXmlOperation(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        final XMLOutputFactory factory = XMLOutputFactory.newInstance();
+        factory.setProperty("com.ctc.wstx.outputValidateStructure", false);
+
+        try {
+            final XMLEventReader eventReader = getXMLReader(encryptedXmlContent);
+            final XMLEventWriter xmlWriter = factory.createXMLEventWriter(decryptedOutputStream);
+            String groupIdentifier = "";
+
+            while(eventReader.hasNext()) {
+                XMLEvent event = eventReader.nextEvent();
+
+                if (isGroupIdentifier(event)) {
+                    groupIdentifier = getGroupIdentifier(eventReader.nextEvent());
+                }
+
+                if (isSensitiveElement(event)) {
+                    xmlWriter.add(updateStartElementEncryptionAttribute(event));
+                    xmlWriter.add(cryptoOperationOnCharacters(eventReader.nextEvent(), groupIdentifier, getPropertyName(event)));
+                } else {
+                    try {
+                        xmlWriter.add(event);
+                    } catch (Exception e) {
+                        throw new RuntimeException("Failed operation on XML content", e);
+                    }
+                }
+            }
+
+            eventReader.close();
+            xmlWriter.flush();
+            xmlWriter.close();
+        } catch (Exception e) {
+            throw new RuntimeException("Failed operation on XML content", e);
+        }
+    }
+
+    /**
+     * Update the StartElement 'encryption' attribute for a sensitive value to add or remove the respective encryption details eg. encryption="aes/gcm/128"
+     * @param xmlEvent A 'sensitive' StartElement that contains the 'encryption' tag attribute
+     * @return The updated StartElement
+     */
+    protected abstract StartElement updateStartElementEncryptionAttribute(final XMLEvent xmlEvent);
+
+    /**
+     * Perform an encrypt or decrypt cryptographic operation on a Characters element
+     * @param xmlEvent A Characters XmlEvent
+     * @param groupIdentifier The XML <identifier/> tag
+     * @return The Characters XmlEvent that has been updated by the cryptographic operation
+     */
+    protected abstract Characters cryptoOperationOnCharacters(final XMLEvent xmlEvent, final String groupIdentifier, final String propertyName);
+
+    private String getGroupIdentifier(final XMLEvent xmlEvent) {
+        if (xmlEvent.isCharacters()) {
+            return xmlEvent.asCharacters().getData();
+        } else {
+            return "";
+        }
+    }
+
+    protected String getPropertyName(final XMLEvent xmlEvent) {
+        return xmlEvent.asStartElement().getName().toString();
+    }
+
+    protected boolean isGroupIdentifier(final XMLEvent xmlEvent) {
+        return xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PARENT_IDENTIFIER);
+    }
+
+    protected boolean isSensitiveElement(final XMLEvent xmlEvent) {
+        return  xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PROPERTY_ELEMENT)
+                && elementHasEncryptionAttribute(xmlEvent.asStartElement());
+    }
+
+    protected XMLEventReader getXMLReader(final InputStream fileStream) throws XMLStreamException, FileNotFoundException {
+        XMLInputFactory xmlInputFactory = XMLInputFactory.newFactory();
+        xmlInputFactory.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+        xmlInputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+
+        return xmlInputFactory.createXMLEventReader(fileStream);
+    }
+
+    private boolean elementHasEncryptionAttribute(final StartElement xmlEvent) {
+        return xmlElementHasAttribute(xmlEvent, ENCRYPTION_ATTRIBUTE_NAME);
+    }
+
+    private boolean xmlElementHasAttribute(final StartElement xmlEvent, final String attributeName) {
+        return !Objects.isNull(xmlEvent.getAttributeByName(new QName(attributeName)));
+    }
+
+    protected StartElement updateElementAttribute(final XMLEvent xmlEvent, final String attributeName, final String attributeValue) {

Review Comment:
   As in, the order within the file?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r953194995


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/xml/XmlCryptoParser.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.xml;
+
+import org.apache.nifi.properties.SensitivePropertyProvider;
+import org.apache.nifi.properties.SensitivePropertyProviderFactory;
+import org.apache.nifi.properties.scheme.ProtectionScheme;
+
+import javax.xml.namespace.QName;
+import javax.xml.stream.XMLEventFactory;
+import javax.xml.stream.XMLEventReader;
+import javax.xml.stream.XMLEventWriter;
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.events.Attribute;
+import javax.xml.stream.events.Characters;
+import javax.xml.stream.events.StartElement;
+import javax.xml.stream.events.XMLEvent;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+public abstract class XmlCryptoParser {
+
+    protected static final String ENCRYPTION_ATTRIBUTE_NAME = "encryption";
+    protected static final String PARENT_IDENTIFIER = "identifier";
+    protected static final String PROPERTY_ELEMENT = "property";
+
+    protected final SensitivePropertyProvider cryptoProvider;
+    protected SensitivePropertyProviderFactory providerFactory;
+
+    public XmlCryptoParser(final SensitivePropertyProviderFactory providerFactory, final ProtectionScheme scheme) {
+        this.providerFactory = providerFactory;
+        cryptoProvider = providerFactory.getProvider(scheme);
+    }
+
+    protected void cryptographicXmlOperation(final InputStream encryptedXmlContent, final OutputStream decryptedOutputStream) {
+        final XMLOutputFactory factory = XMLOutputFactory.newInstance();
+        factory.setProperty("com.ctc.wstx.outputValidateStructure", false);
+
+        try {
+            final XMLEventReader eventReader = getXMLReader(encryptedXmlContent);
+            final XMLEventWriter xmlWriter = factory.createXMLEventWriter(decryptedOutputStream);
+            String groupIdentifier = "";
+
+            while(eventReader.hasNext()) {
+                XMLEvent event = eventReader.nextEvent();
+
+                if (isGroupIdentifier(event)) {
+                    groupIdentifier = getGroupIdentifier(eventReader.nextEvent());
+                }
+
+                if (isSensitiveElement(event)) {
+                    xmlWriter.add(updateStartElementEncryptionAttribute(event));
+                    xmlWriter.add(cryptoOperationOnCharacters(eventReader.nextEvent(), groupIdentifier, getPropertyName(event)));
+                } else {
+                    try {
+                        xmlWriter.add(event);
+                    } catch (Exception e) {
+                        throw new RuntimeException("Failed operation on XML content", e);
+                    }
+                }
+            }
+
+            eventReader.close();
+            xmlWriter.flush();
+            xmlWriter.close();
+        } catch (Exception e) {
+            throw new RuntimeException("Failed operation on XML content", e);
+        }
+    }
+
+    /**
+     * Update the StartElement 'encryption' attribute for a sensitive value to add or remove the respective encryption details eg. encryption="aes/gcm/128"
+     * @param xmlEvent A 'sensitive' StartElement that contains the 'encryption' tag attribute
+     * @return The updated StartElement
+     */
+    protected abstract StartElement updateStartElementEncryptionAttribute(final XMLEvent xmlEvent);
+
+    /**
+     * Perform an encrypt or decrypt cryptographic operation on a Characters element
+     * @param xmlEvent A Characters XmlEvent
+     * @param groupIdentifier The XML <identifier/> tag
+     * @return The Characters XmlEvent that has been updated by the cryptographic operation
+     */
+    protected abstract Characters cryptoOperationOnCharacters(final XMLEvent xmlEvent, final String groupIdentifier, final String propertyName);
+
+    private String getGroupIdentifier(final XMLEvent xmlEvent) {
+        if (xmlEvent.isCharacters()) {
+            return xmlEvent.asCharacters().getData();
+        } else {
+            return "";
+        }
+    }
+
+    protected String getPropertyName(final XMLEvent xmlEvent) {
+        return xmlEvent.asStartElement().getName().toString();
+    }
+
+    protected boolean isGroupIdentifier(final XMLEvent xmlEvent) {
+        return xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PARENT_IDENTIFIER);
+    }
+
+    protected boolean isSensitiveElement(final XMLEvent xmlEvent) {
+        return  xmlEvent.isStartElement()
+                && xmlEvent.asStartElement().getName().toString().equals(PROPERTY_ELEMENT)
+                && elementHasEncryptionAttribute(xmlEvent.asStartElement());
+    }
+
+    protected XMLEventReader getXMLReader(final InputStream fileStream) throws XMLStreamException, FileNotFoundException {
+        XMLInputFactory xmlInputFactory = XMLInputFactory.newFactory();
+        xmlInputFactory.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+        xmlInputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+
+        return xmlInputFactory.createXMLEventReader(fileStream);
+    }
+
+    private boolean elementHasEncryptionAttribute(final StartElement xmlEvent) {
+        return xmlElementHasAttribute(xmlEvent, ENCRYPTION_ATTRIBUTE_NAME);
+    }
+
+    private boolean xmlElementHasAttribute(final StartElement xmlEvent, final String attributeName) {
+        return !Objects.isNull(xmlEvent.getAttributeByName(new QName(attributeName)));
+    }
+
+    protected StartElement updateElementAttribute(final XMLEvent xmlEvent, final String attributeName, final String attributeValue) {

Review Comment:
   Correct, yes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] thenatog commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
thenatog commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r959741034


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/ConfigSubcommand.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.util.console;
+
+import org.apache.nifi.PropertyEncryptorMain;
+import org.apache.nifi.properties.scheme.PropertyProtectionScheme;
+import org.apache.nifi.util.console.utils.BaseCommandParameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+
+@CommandLine.Command(name = "config",
+        description = "Operate on application configs",
+        usageHelpWidth=140
+)
+class ConfigSubcommand extends BaseCommandParameters implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConfigSubcommand.class);
+
+    @CommandLine.ParentCommand
+    private DefaultCLIOptions parent;
+
+    @CommandLine.Parameters(description="The encryption scheme to use, from one of the following schemes: [@|bold ${COMPLETION-CANDIDATES}|@]")
+    PropertyProtectionScheme scheme;
+
+    @Override
+    public void run() {
+        final PropertyEncryptorMain propertyEncryptorMain = new PropertyEncryptorMain(baseDirectory, passphrase);

Review Comment:
   What would be a good way to handle some of the common methods across the different commands like retrieving/storing keys, getting the properties loader and configuration file resolver etc?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6273: NIFI-9953 - The config encryption tool is too complicated to use and can be simplified

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6273:
URL: https://github.com/apache/nifi/pull/6273#discussion_r1033720118


##########
nifi-toolkit/nifi-property-encryptor-tool/src/main/java/org/apache/nifi/util/console/utils/SchemeCandidates.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.util.console.utils;
+
+import org.apache.nifi.properties.scheme.StandardProtectionSchemeResolver;
+
+import java.util.ArrayList;
+
+public class SchemeCandidates extends ArrayList<String> {
+    SchemeCandidates() {
+        super(new StandardProtectionSchemeResolver().getSupportedProtectionSchemes());

Review Comment:
   There is no existing way to enumerate the Protection Scheme paths, so streaming the array of enum values and mapping to `getPath` in this class should work.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org