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 2021/01/13 16:25:00 UTC

[GitHub] [nifi] exceptionfactory commented on a change in pull request #4753: NIFI-7356 - Enable TLS for embedded Zookeeper when NiFi has TLS enabled

exceptionfactory commented on a change in pull request #4753:
URL: https://github.com/apache/nifi/pull/4753#discussion_r556642464



##########
File path: nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
##########
@@ -1540,6 +1540,12 @@ public boolean isZooKeeperTlsConfigurationPresent() {
             && getProperty(NiFiProperties.ZOOKEEPER_SECURITY_TRUSTSTORE_PASSWD) != null;
     }
 
+    public boolean isTlsConfigurationPresent() {
+        return StringUtils.isNotBlank(getProperty(NiFiProperties.SECURITY_KEYSTORE))
+            && getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD) != null

Review comment:
       Is there a reason for just checking whether the keystore password is not null, as opposed to checking whether it is not blank?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
##########
@@ -36,11 +36,33 @@
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Properties;
 
 public class ZooKeeperStateServer extends ZooKeeperServerMain {
     private static final Logger logger = LoggerFactory.getLogger(ZooKeeperStateServer.class);
 
+    static final int MIN_AVAILABLE_PORT = 2288;
+    static final String SERVER_CNXN_FACTORY = "org.apache.zookeeper.server.NettyServerCnxnFactory";

Review comment:
       Is there a reason for using the string class name?  Recommend replacing references with the actual class value and using `Class.getName()`.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
##########
@@ -36,11 +36,33 @@
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Properties;
 
 public class ZooKeeperStateServer extends ZooKeeperServerMain {
     private static final Logger logger = LoggerFactory.getLogger(ZooKeeperStateServer.class);
 
+    static final int MIN_AVAILABLE_PORT = 2288;
+    static final String SERVER_CNXN_FACTORY = "org.apache.zookeeper.server.NettyServerCnxnFactory";
+    static final String ZOOKEEPER_SSL_QUORUM = "sslQuorum";
+    static final String ZOOKEEPER_PORT_UNIFICATION = "portUnification";
+
+    static final Map<String, String> ZOOKEEPER_TLS_TO_NIFI_PROPERTIES = new HashMap<String, String>() {{
+        put("keyStore.location", "security.keystore");
+        put("keyStore.password", "security.keystorePasswd");
+        put("keyStore.type", "security.keystoreType");
+        put("trustStore.location", "security.truststore");
+        put("trustStore.password", "security.truststorePasswd");
+        put("trustStore.type", "security.truststoreType");

Review comment:
       The `NiFiProperites` class has static variables for the NiFi property names, and the ZooKeeper `X509Util` class has methods to get the ZooKeeper property names.  Would it be possible to refactor this map and other property name references to use those variables and methods instead of repeating the string values?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
##########
@@ -198,6 +233,110 @@ public static ZooKeeperStateServer create(final NiFiProperties properties) throw
             zkProperties.load(bis);
         }
 
-        return new ZooKeeperStateServer(zkProperties);
+        return new ZooKeeperStateServer(reconcileProperties(properties, zkProperties));
+    }
+
+    /**
+     * Reconcile properties between the nifi.properties and zookeeper.properties (zoo.cfg) files. Most of the ZooKeeper server properties are derived from
+     * the zookeeper.properties file, while the TLS key/truststore properties are taken from nifi.properties.
+     * @param niFiProperties NiFiProperties file containing ZooKeeper client and TLS configuration
+     * @param zkProperties The zookeeper.properties file containing Zookeeper server configuration
+     * @return A reconciled QuorumPeerConfig which will include TLS properties set if they are available.
+     * @throws IOException If configuration files fail to parse.
+     * @throws ConfigException If secure configuration is not as expected. Check administration documentation.
+     */
+    private static QuorumPeerConfig reconcileProperties(NiFiProperties niFiProperties, Properties zkProperties) throws IOException, ConfigException {
+        QuorumPeerConfig peerConfig = new QuorumPeerConfig();
+        peerConfig.parseProperties(zkProperties);
+
+        // If secureClientPortAddress is set but no TLS config is set, fail to start.
+        final boolean isTLSConfigPresent = niFiProperties.isTlsConfigurationPresent() || niFiProperties.isZooKeeperTlsConfigurationPresent();
+        if (peerConfig.getSecureClientPortAddress() != null && !isTLSConfigPresent) {
+            throw new ConfigException(
+                    String.format("Property secureClientPort was set in %s but there was no TLS config present in nifi.properties",
+                    niFiProperties.getProperty(NiFiProperties.STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES)));
+        }
+
+        // If this is an insecure NiFi no changes are needed:
+        if (!isTLSConfigPresent) {
+            logger.info("ZooKeeper is not secure because appropriate TLS configuration was not provided. Please refer to administration guide.");
+            return peerConfig;
+        }
+
+        // Otherwise the following sets secure TLS settings for embedded Zookeeper
+
+        // Remove plaintext client ports and addresses and warn if set, see NIFI-7203:
+        InetSocketAddress clientPort = peerConfig.getClientPortAddress();
+        if (clientPort != null) {
+            zkProperties.remove("clientPort");
+            zkProperties.remove("clientPortAddress");
+            logger.warn("Invalid configuration detected: secure NiFi with embedded ZooKeeper configured for insecure connections. " +
+                    "Removed insecure port from embedded ZooKeeper configuration to deactivate insecure connections.");
+        }
+
+        // Set the TLS properties, preferring ZK TLS from nifi.properties over the default TLS properties.
+        for (Map.Entry<String, String> propertyMapping : ZOOKEEPER_TLS_TO_NIFI_PROPERTIES.entrySet()) {
+            String preferredValue = getPreferredTLSProperty(niFiProperties, propertyMapping.getValue());
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_QUORUM_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+        }
+
+        // Set TLS client port:
+        zkProperties.setProperty("secureClientPort", getSecurePort(peerConfig));
+
+        // Set the required connection factory for TLS
+        final String cnxnPropKey = "serverCnxnFactory";
+        zkProperties.setProperty(cnxnPropKey, SERVER_CNXN_FACTORY);
+        zkProperties.setProperty(ZOOKEEPER_SSL_QUORUM, "true");
+
+        // Port unification allows both secure and insecure connections - setting to false means only secure connections will be allowed.
+        zkProperties.setProperty(ZOOKEEPER_PORT_UNIFICATION, "false");
+
+        // Recreate and reload the adjusted properties to ensure they're still valid for ZK:
+        peerConfig = new QuorumPeerConfig();
+        peerConfig.parseProperties(zkProperties);
+        return peerConfig;
+    }
+
+    private static String getSecurePort(QuorumPeerConfig peerConfig) throws ConfigException {
+        final InetSocketAddress secureClientAddress = peerConfig.getSecureClientPortAddress();
+        String secureClientPort = null;
+
+        if (secureClientAddress != null && String.valueOf(secureClientAddress.getPort()) != null) {
+            secureClientPort = String.valueOf(secureClientAddress.getPort());
+            logger.info("Secure client port retrieved from ZooKeeper configuration: {}", secureClientPort);

Review comment:
       Should this be changed to `debug()` instead of `info()`

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
##########
@@ -198,6 +233,110 @@ public static ZooKeeperStateServer create(final NiFiProperties properties) throw
             zkProperties.load(bis);
         }
 
-        return new ZooKeeperStateServer(zkProperties);
+        return new ZooKeeperStateServer(reconcileProperties(properties, zkProperties));
+    }
+
+    /**
+     * Reconcile properties between the nifi.properties and zookeeper.properties (zoo.cfg) files. Most of the ZooKeeper server properties are derived from
+     * the zookeeper.properties file, while the TLS key/truststore properties are taken from nifi.properties.
+     * @param niFiProperties NiFiProperties file containing ZooKeeper client and TLS configuration
+     * @param zkProperties The zookeeper.properties file containing Zookeeper server configuration
+     * @return A reconciled QuorumPeerConfig which will include TLS properties set if they are available.
+     * @throws IOException If configuration files fail to parse.
+     * @throws ConfigException If secure configuration is not as expected. Check administration documentation.
+     */
+    private static QuorumPeerConfig reconcileProperties(NiFiProperties niFiProperties, Properties zkProperties) throws IOException, ConfigException {
+        QuorumPeerConfig peerConfig = new QuorumPeerConfig();
+        peerConfig.parseProperties(zkProperties);
+
+        // If secureClientPortAddress is set but no TLS config is set, fail to start.
+        final boolean isTLSConfigPresent = niFiProperties.isTlsConfigurationPresent() || niFiProperties.isZooKeeperTlsConfigurationPresent();
+        if (peerConfig.getSecureClientPortAddress() != null && !isTLSConfigPresent) {
+            throw new ConfigException(
+                    String.format("Property secureClientPort was set in %s but there was no TLS config present in nifi.properties",
+                    niFiProperties.getProperty(NiFiProperties.STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES)));
+        }
+
+        // If this is an insecure NiFi no changes are needed:
+        if (!isTLSConfigPresent) {
+            logger.info("ZooKeeper is not secure because appropriate TLS configuration was not provided. Please refer to administration guide.");
+            return peerConfig;
+        }
+
+        // Otherwise the following sets secure TLS settings for embedded Zookeeper
+
+        // Remove plaintext client ports and addresses and warn if set, see NIFI-7203:
+        InetSocketAddress clientPort = peerConfig.getClientPortAddress();
+        if (clientPort != null) {
+            zkProperties.remove("clientPort");
+            zkProperties.remove("clientPortAddress");
+            logger.warn("Invalid configuration detected: secure NiFi with embedded ZooKeeper configured for insecure connections. " +
+                    "Removed insecure port from embedded ZooKeeper configuration to deactivate insecure connections.");
+        }
+
+        // Set the TLS properties, preferring ZK TLS from nifi.properties over the default TLS properties.
+        for (Map.Entry<String, String> propertyMapping : ZOOKEEPER_TLS_TO_NIFI_PROPERTIES.entrySet()) {
+            String preferredValue = getPreferredTLSProperty(niFiProperties, propertyMapping.getValue());
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_QUORUM_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+        }
+
+        // Set TLS client port:
+        zkProperties.setProperty("secureClientPort", getSecurePort(peerConfig));
+
+        // Set the required connection factory for TLS
+        final String cnxnPropKey = "serverCnxnFactory";
+        zkProperties.setProperty(cnxnPropKey, SERVER_CNXN_FACTORY);
+        zkProperties.setProperty(ZOOKEEPER_SSL_QUORUM, "true");
+
+        // Port unification allows both secure and insecure connections - setting to false means only secure connections will be allowed.
+        zkProperties.setProperty(ZOOKEEPER_PORT_UNIFICATION, "false");
+
+        // Recreate and reload the adjusted properties to ensure they're still valid for ZK:
+        peerConfig = new QuorumPeerConfig();
+        peerConfig.parseProperties(zkProperties);
+        return peerConfig;
+    }
+
+    private static String getSecurePort(QuorumPeerConfig peerConfig) throws ConfigException {
+        final InetSocketAddress secureClientAddress = peerConfig.getSecureClientPortAddress();
+        String secureClientPort = null;
+
+        if (secureClientAddress != null && String.valueOf(secureClientAddress.getPort()) != null) {

Review comment:
       `InetSocketAddress.getPort()` should always return an `int`, so checking for `String.valueOf(secureClientAddress.getPort())` should never be null.  Is it necessary to check for a value between 1024 and 65535?

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/server/TestZooKeeperStateServerConfigurations.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.controller.state.server;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+// This class tests the behaviors involved with the ZooKeeperStateServer::create method.  The servers are not started,
+// and TLS connections are not used.
+public class TestZooKeeperStateServerConfigurations {
+    private static final String KEY_STORE = getPath( "keystore.jks");
+    private static final String TRUST_STORE = getPath( "truststore.jks");
+    private static final String INSECURE_ZOOKEEPER_PROPS = getPath( "insecure.zookeeper.properties");
+    private static final String COMPLETE_ZOOKEEPER_PROPS = getPath( "complete.zookeeper.properties");
+    private static final String SECURE_ZOOKEEPER_PROPS = getPath( "secure.zookeeper.properties");

Review comment:
       The previous method calls have an extra space prior to the path string.

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
##########
@@ -198,6 +233,110 @@ public static ZooKeeperStateServer create(final NiFiProperties properties) throw
             zkProperties.load(bis);
         }
 
-        return new ZooKeeperStateServer(zkProperties);
+        return new ZooKeeperStateServer(reconcileProperties(properties, zkProperties));
+    }
+
+    /**
+     * Reconcile properties between the nifi.properties and zookeeper.properties (zoo.cfg) files. Most of the ZooKeeper server properties are derived from
+     * the zookeeper.properties file, while the TLS key/truststore properties are taken from nifi.properties.
+     * @param niFiProperties NiFiProperties file containing ZooKeeper client and TLS configuration
+     * @param zkProperties The zookeeper.properties file containing Zookeeper server configuration
+     * @return A reconciled QuorumPeerConfig which will include TLS properties set if they are available.
+     * @throws IOException If configuration files fail to parse.
+     * @throws ConfigException If secure configuration is not as expected. Check administration documentation.
+     */
+    private static QuorumPeerConfig reconcileProperties(NiFiProperties niFiProperties, Properties zkProperties) throws IOException, ConfigException {
+        QuorumPeerConfig peerConfig = new QuorumPeerConfig();
+        peerConfig.parseProperties(zkProperties);
+
+        // If secureClientPortAddress is set but no TLS config is set, fail to start.
+        final boolean isTLSConfigPresent = niFiProperties.isTlsConfigurationPresent() || niFiProperties.isZooKeeperTlsConfigurationPresent();
+        if (peerConfig.getSecureClientPortAddress() != null && !isTLSConfigPresent) {
+            throw new ConfigException(
+                    String.format("Property secureClientPort was set in %s but there was no TLS config present in nifi.properties",
+                    niFiProperties.getProperty(NiFiProperties.STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES)));
+        }
+
+        // If this is an insecure NiFi no changes are needed:
+        if (!isTLSConfigPresent) {
+            logger.info("ZooKeeper is not secure because appropriate TLS configuration was not provided. Please refer to administration guide.");
+            return peerConfig;
+        }
+
+        // Otherwise the following sets secure TLS settings for embedded Zookeeper
+
+        // Remove plaintext client ports and addresses and warn if set, see NIFI-7203:
+        InetSocketAddress clientPort = peerConfig.getClientPortAddress();
+        if (clientPort != null) {
+            zkProperties.remove("clientPort");
+            zkProperties.remove("clientPortAddress");
+            logger.warn("Invalid configuration detected: secure NiFi with embedded ZooKeeper configured for insecure connections. " +
+                    "Removed insecure port from embedded ZooKeeper configuration to deactivate insecure connections.");
+        }
+
+        // Set the TLS properties, preferring ZK TLS from nifi.properties over the default TLS properties.
+        for (Map.Entry<String, String> propertyMapping : ZOOKEEPER_TLS_TO_NIFI_PROPERTIES.entrySet()) {
+            String preferredValue = getPreferredTLSProperty(niFiProperties, propertyMapping.getValue());
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_QUORUM_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+        }
+
+        // Set TLS client port:
+        zkProperties.setProperty("secureClientPort", getSecurePort(peerConfig));
+
+        // Set the required connection factory for TLS
+        final String cnxnPropKey = "serverCnxnFactory";
+        zkProperties.setProperty(cnxnPropKey, SERVER_CNXN_FACTORY);

Review comment:
       Recommend replacing `SERVER_CNXN_FACTORY` with `Class.getName()`:
   ```suggestion
           zkProperties.setProperty(cnxnPropKey, NettyServerCnxnFactory.class.getName());
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
##########
@@ -198,6 +233,110 @@ public static ZooKeeperStateServer create(final NiFiProperties properties) throw
             zkProperties.load(bis);
         }
 
-        return new ZooKeeperStateServer(zkProperties);
+        return new ZooKeeperStateServer(reconcileProperties(properties, zkProperties));
+    }
+
+    /**
+     * Reconcile properties between the nifi.properties and zookeeper.properties (zoo.cfg) files. Most of the ZooKeeper server properties are derived from
+     * the zookeeper.properties file, while the TLS key/truststore properties are taken from nifi.properties.
+     * @param niFiProperties NiFiProperties file containing ZooKeeper client and TLS configuration
+     * @param zkProperties The zookeeper.properties file containing Zookeeper server configuration
+     * @return A reconciled QuorumPeerConfig which will include TLS properties set if they are available.
+     * @throws IOException If configuration files fail to parse.
+     * @throws ConfigException If secure configuration is not as expected. Check administration documentation.
+     */
+    private static QuorumPeerConfig reconcileProperties(NiFiProperties niFiProperties, Properties zkProperties) throws IOException, ConfigException {
+        QuorumPeerConfig peerConfig = new QuorumPeerConfig();
+        peerConfig.parseProperties(zkProperties);
+
+        // If secureClientPortAddress is set but no TLS config is set, fail to start.
+        final boolean isTLSConfigPresent = niFiProperties.isTlsConfigurationPresent() || niFiProperties.isZooKeeperTlsConfigurationPresent();
+        if (peerConfig.getSecureClientPortAddress() != null && !isTLSConfigPresent) {
+            throw new ConfigException(
+                    String.format("Property secureClientPort was set in %s but there was no TLS config present in nifi.properties",
+                    niFiProperties.getProperty(NiFiProperties.STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES)));
+        }
+
+        // If this is an insecure NiFi no changes are needed:
+        if (!isTLSConfigPresent) {
+            logger.info("ZooKeeper is not secure because appropriate TLS configuration was not provided. Please refer to administration guide.");
+            return peerConfig;
+        }
+
+        // Otherwise the following sets secure TLS settings for embedded Zookeeper
+
+        // Remove plaintext client ports and addresses and warn if set, see NIFI-7203:
+        InetSocketAddress clientPort = peerConfig.getClientPortAddress();
+        if (clientPort != null) {
+            zkProperties.remove("clientPort");
+            zkProperties.remove("clientPortAddress");
+            logger.warn("Invalid configuration detected: secure NiFi with embedded ZooKeeper configured for insecure connections. " +
+                    "Removed insecure port from embedded ZooKeeper configuration to deactivate insecure connections.");
+        }
+
+        // Set the TLS properties, preferring ZK TLS from nifi.properties over the default TLS properties.
+        for (Map.Entry<String, String> propertyMapping : ZOOKEEPER_TLS_TO_NIFI_PROPERTIES.entrySet()) {
+            String preferredValue = getPreferredTLSProperty(niFiProperties, propertyMapping.getValue());
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_QUORUM_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+        }
+
+        // Set TLS client port:
+        zkProperties.setProperty("secureClientPort", getSecurePort(peerConfig));
+
+        // Set the required connection factory for TLS
+        final String cnxnPropKey = "serverCnxnFactory";
+        zkProperties.setProperty(cnxnPropKey, SERVER_CNXN_FACTORY);
+        zkProperties.setProperty(ZOOKEEPER_SSL_QUORUM, "true");
+
+        // Port unification allows both secure and insecure connections - setting to false means only secure connections will be allowed.
+        zkProperties.setProperty(ZOOKEEPER_PORT_UNIFICATION, "false");

Review comment:
       Recommend replacing with `Boolean.FALSE.toString()`:
   ```suggestion
           zkProperties.setProperty(ZOOKEEPER_PORT_UNIFICATION, Boolean.FALSE.toString());
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/server/ZooKeeperStateServer.java
##########
@@ -198,6 +233,110 @@ public static ZooKeeperStateServer create(final NiFiProperties properties) throw
             zkProperties.load(bis);
         }
 
-        return new ZooKeeperStateServer(zkProperties);
+        return new ZooKeeperStateServer(reconcileProperties(properties, zkProperties));
+    }
+
+    /**
+     * Reconcile properties between the nifi.properties and zookeeper.properties (zoo.cfg) files. Most of the ZooKeeper server properties are derived from
+     * the zookeeper.properties file, while the TLS key/truststore properties are taken from nifi.properties.
+     * @param niFiProperties NiFiProperties file containing ZooKeeper client and TLS configuration
+     * @param zkProperties The zookeeper.properties file containing Zookeeper server configuration
+     * @return A reconciled QuorumPeerConfig which will include TLS properties set if they are available.
+     * @throws IOException If configuration files fail to parse.
+     * @throws ConfigException If secure configuration is not as expected. Check administration documentation.
+     */
+    private static QuorumPeerConfig reconcileProperties(NiFiProperties niFiProperties, Properties zkProperties) throws IOException, ConfigException {
+        QuorumPeerConfig peerConfig = new QuorumPeerConfig();
+        peerConfig.parseProperties(zkProperties);
+
+        // If secureClientPortAddress is set but no TLS config is set, fail to start.
+        final boolean isTLSConfigPresent = niFiProperties.isTlsConfigurationPresent() || niFiProperties.isZooKeeperTlsConfigurationPresent();
+        if (peerConfig.getSecureClientPortAddress() != null && !isTLSConfigPresent) {
+            throw new ConfigException(
+                    String.format("Property secureClientPort was set in %s but there was no TLS config present in nifi.properties",
+                    niFiProperties.getProperty(NiFiProperties.STATE_MANAGEMENT_ZOOKEEPER_PROPERTIES)));
+        }
+
+        // If this is an insecure NiFi no changes are needed:
+        if (!isTLSConfigPresent) {
+            logger.info("ZooKeeper is not secure because appropriate TLS configuration was not provided. Please refer to administration guide.");
+            return peerConfig;
+        }
+
+        // Otherwise the following sets secure TLS settings for embedded Zookeeper
+
+        // Remove plaintext client ports and addresses and warn if set, see NIFI-7203:
+        InetSocketAddress clientPort = peerConfig.getClientPortAddress();
+        if (clientPort != null) {
+            zkProperties.remove("clientPort");
+            zkProperties.remove("clientPortAddress");
+            logger.warn("Invalid configuration detected: secure NiFi with embedded ZooKeeper configured for insecure connections. " +
+                    "Removed insecure port from embedded ZooKeeper configuration to deactivate insecure connections.");
+        }
+
+        // Set the TLS properties, preferring ZK TLS from nifi.properties over the default TLS properties.
+        for (Map.Entry<String, String> propertyMapping : ZOOKEEPER_TLS_TO_NIFI_PROPERTIES.entrySet()) {
+            String preferredValue = getPreferredTLSProperty(niFiProperties, propertyMapping.getValue());
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+            zkProperties.setProperty(String.join(".", ZOOKEEPER_QUORUM_TLS_PREFIX, propertyMapping.getKey()), preferredValue);
+        }
+
+        // Set TLS client port:
+        zkProperties.setProperty("secureClientPort", getSecurePort(peerConfig));
+
+        // Set the required connection factory for TLS
+        final String cnxnPropKey = "serverCnxnFactory";
+        zkProperties.setProperty(cnxnPropKey, SERVER_CNXN_FACTORY);
+        zkProperties.setProperty(ZOOKEEPER_SSL_QUORUM, "true");

Review comment:
       Recommend replacing with `Boolean.TRUE.toString()`:
   ```suggestion
           zkProperties.setProperty(ZOOKEEPER_SSL_QUORUM, Boolean.TRUE.toString());
   ```

##########
File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/state/server/ITZooKeeperStateServerTLS.java
##########
@@ -0,0 +1,415 @@
+/*
+ * 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.controller.state.server;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.retry.RetryOneTime;
+import org.apache.curator.utils.DefaultZookeeperFactory;
+import org.apache.curator.utils.ZookeeperFactory;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.nifi.controller.cluster.SecureClientZooKeeperFactory;
+import org.apache.nifi.controller.cluster.ZooKeeperClientConfig;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.nifi.leader.election.ITSecureClientZooKeeperFactory.createSecureClientProperties;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+// Testing setting up a ZooKeeperStateServer with TLS
+public class ITZooKeeperStateServerTLS {
+    private static final String KEY_STORE = getPath("keystore.jks");
+    private static final String TRUST_STORE = getPath("truststore.jks");
+    private static final String STORE_TYPE = "JKS";
+    private static final String INSECURE_ZOOKEEPER_PROPS = getPath("insecure.zookeeper.properties");
+    private static final String PARTIAL_ZOOKEEPER_PROPS = getPath("partial.zookeeper.properties");
+    private static final String COMPLETE_ZOOKEEPER_PROPS = getPath("complete.zookeeper.properties");
+    private static final String SECURE_ZOOKEEPER_PROPS = getPath("secure.zookeeper.properties");
+    private static final String ZOOKEEPER_PROPERTIES_FILE_KEY = "nifi.state.management.embedded.zookeeper.properties";
+    private static final String ZOOKEEPER_CNXN_FACTORY = "org.apache.zookeeper.server.NettyServerCnxnFactory";
+    private static final String CONNECT_STRING = "localhost:2281";
+    private static final String QUORUM_CONNECT_STRING = "node0.apache.org:2281,node1.apache.org:2281";
+
+    private static final Map<String, String> INSECURE_NIFI_PROPS = new HashMap<String, String>() {{
+        put(ZOOKEEPER_PROPERTIES_FILE_KEY, INSECURE_ZOOKEEPER_PROPS);
+        put(NiFiProperties.WEB_HTTP_HOST, "localhost");
+        put(NiFiProperties.WEB_HTTP_PORT, "8080");
+        put(NiFiProperties.ZOOKEEPER_CLIENT_SECURE, "false");
+    }};
+
+    private static final String TEST_PASSWORD = "passwordpassword";
+
+    private static final Map<String, String> SECURE_NIFI_PROPS = new HashMap<String, String>() {{
+        put(ZOOKEEPER_PROPERTIES_FILE_KEY, SECURE_ZOOKEEPER_PROPS);
+        put("nifi.web.https.port", "8443");
+        put("nifi.security.keystore", KEY_STORE);
+        put("nifi.security.keystoreType", STORE_TYPE);
+        put("nifi.security.keystorePasswd", TEST_PASSWORD);
+        put("nifi.security.truststore", TRUST_STORE);
+        put("nifi.security.truststoreType", STORE_TYPE);
+        put("nifi.security.truststorePasswd", TEST_PASSWORD);

Review comment:
       See comments regarding replacing strings with references to `NiFiProperties` values.




----------------------------------------------------------------
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.

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